From f167002522d50eefb261c8ba2d66a23b781a38c4 Mon Sep 17 00:00:00 2001 From: Tom van Bussel Date: Tue, 29 Sep 2020 13:05:33 +0200 Subject: [PATCH 01/59] [SPARK-32901][CORE] Do not allocate memory while spilling UnsafeExternalSorter ### What changes were proposed in this pull request? This PR changes `UnsafeExternalSorter` to no longer allocate any memory while spilling. In particular it removes the allocation of a new pointer array in `UnsafeInMemorySorter`. Instead the new pointer array is allocated whenever the next record is inserted into the sorter. ### Why are the changes needed? Without this change the `UnsafeExternalSorter` could throw an OOM while spilling. The following sequence of events would have triggered an OOM: 1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one. 2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested. 3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`. 4. `UnsafeInMemorySorter` frees the old pointer array, and tries to allocate a new small pointer array. 5. `TaskMemoryManager` tries to allocate the memory backing the small array using `MemoryManager`, but `MemoryManager` is unwilling to give it any memory, as the `TaskMemoryManager` is still holding on to the memory it got for the new large array. 6. `TaskMemoryManager` again asks `UnsafeExternalSorter` to spill, but this time there is nothing to spill. 7. `UnsafeInMemorySorter` receives less memory than it requested, and causes a `SparkOutOfMemoryError` to be thrown, which causes the current task to fail. With the changes in the PR the following will happen instead: 1. `UnsafeExternalSorter` runs out of space in its pointer array and attempts to allocate a new large array to replace the old one. 2. `TaskMemoryManager` tries to allocate the memory backing the new large array using `MemoryManager`, but `MemoryManager` is only willing to return most but not all of the memory requested. 3. `TaskMemoryManager` asks `UnsafeExternalSorter` to spill, which causes `UnsafeExternalSorter` to spill the current run to disk, to free its record pages and to reset its `UnsafeInMemorySorter`. 4. `UnsafeInMemorySorter` frees the old pointer array. 5. `TaskMemoryManager` returns control to `UnsafeExternalSorter.growPointerArrayIfNecessary` (either by returning the the new large array or by throwing a `SparkOutOfMemoryError`). 6. `UnsafeExternalSorter` either frees the new large array or it ignores the `SparkOutOfMemoryError` depending on what happened in the previous step. 7. `UnsafeExternalSorter` successfully allocates a new small pointer array and operation continues as normal. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests were added in `UnsafeExternalSorterSuite` and `UnsafeInMemorySorterSuite`. Closes #29785 from tomvanbussel/SPARK-32901. Authored-by: Tom van Bussel Signed-off-by: herman --- .../unsafe/sort/UnsafeExternalSorter.java | 96 ++++++++++++++----- .../unsafe/sort/UnsafeInMemorySorter.java | 55 +++++------ .../sort/UnsafeExternalSorterSuite.java | 46 ++++----- .../sort/UnsafeInMemorySorterSuite.java | 40 ++++---- .../spark/memory/TestMemoryManager.scala | 8 ++ 5 files changed, 143 insertions(+), 102 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index e4a882d609fc2..dda8ed4c239ae 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -203,6 +203,10 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } if (inMemSorter == null || inMemSorter.numRecords() <= 0) { + // There could still be some memory allocated when there are no records in the in-memory + // sorter. We will not spill it however, to ensure that we can always process at least one + // record before spilling. See the comments in `allocateMemoryForRecordIfNecessary` for why + // this is necessary. return 0L; } @@ -224,7 +228,7 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { // Note that this is more-or-less going to be a multiple of the page size, so wasted space in // pages will currently be counted as memory spilled even though that space isn't actually // written to disk. This also counts the space needed to store the sorter's pointer array. - inMemSorter.reset(); + inMemSorter.freeMemory(); // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the // records. Otherwise, if the task is over allocated memory, then without freeing the memory // pages, we might not be able to get memory for the pointer array. @@ -325,7 +329,7 @@ public void cleanupResources() { deleteSpillFiles(); freeMemory(); if (inMemSorter != null) { - inMemSorter.free(); + inMemSorter.freeMemory(); inMemSorter = null; } } @@ -339,40 +343,53 @@ public void cleanupResources() { private void growPointerArrayIfNecessary() throws IOException { assert(inMemSorter != null); if (!inMemSorter.hasSpaceForAnotherRecord()) { + if (inMemSorter.numRecords() <= 0) { + // Spilling was triggered just before this method was called. The pointer array was freed + // during the spill, so a new pointer array needs to be allocated here. + LongArray array = allocateArray(inMemSorter.getInitialSize()); + inMemSorter.expandPointerArray(array); + return; + } + long used = inMemSorter.getMemoryUsage(); - LongArray array; + LongArray array = null; try { // could trigger spilling array = allocateArray(used / 8 * 2); } catch (TooLargePageException e) { // The pointer array is too big to fix in a single page, spill. spill(); - return; } catch (SparkOutOfMemoryError e) { - // should have trigger spilling - if (!inMemSorter.hasSpaceForAnotherRecord()) { + if (inMemSorter.numRecords() > 0) { logger.error("Unable to grow the pointer array"); throw e; } - return; + // The new array could not be allocated, but that is not an issue as it is longer needed, + // as all records were spilled. } - // check if spilling is triggered or not - if (inMemSorter.hasSpaceForAnotherRecord()) { - freeArray(array); - } else { - inMemSorter.expandPointerArray(array); + + if (inMemSorter.numRecords() <= 0) { + // Spilling was triggered while trying to allocate the new array. + if (array != null) { + // We succeeded in allocating the new array, but, since all records were spilled, a + // smaller array would also suffice. + freeArray(array); + } + // The pointer array was freed during the spill, so a new pointer array needs to be + // allocated here. + array = allocateArray(inMemSorter.getInitialSize()); } + inMemSorter.expandPointerArray(array); } } /** - * Allocates more memory in order to insert an additional record. This will request additional - * memory from the memory manager and spill if the requested memory can not be obtained. + * Allocates an additional page in order to insert an additional record. This will request + * additional memory from the memory manager and spill if the requested memory can not be + * obtained. * * @param required the required space in the data page, in bytes, including space for storing - * the record size. This must be less than or equal to the page size (records - * that exceed the page size are handled via a different code path which uses - * special overflow pages). + * the record size. */ private void acquireNewPageIfNecessary(int required) { if (currentPage == null || @@ -384,6 +401,37 @@ private void acquireNewPageIfNecessary(int required) { } } + /** + * Allocates more memory in order to insert an additional record. This will request additional + * memory from the memory manager and spill if the requested memory can not be obtained. + * + * @param required the required space in the data page, in bytes, including space for storing + * the record size. + */ + private void allocateMemoryForRecordIfNecessary(int required) throws IOException { + // Step 1: + // Ensure that the pointer array has space for another record. This may cause a spill. + growPointerArrayIfNecessary(); + // Step 2: + // Ensure that the last page has space for another record. This may cause a spill. + acquireNewPageIfNecessary(required); + // Step 3: + // The allocation in step 2 could have caused a spill, which would have freed the pointer + // array allocated in step 1. Therefore we need to check again whether we have to allocate + // a new pointer array. + // + // If the allocation in this step causes a spill event then it will not cause the page + // allocated in the previous step to be freed. The function `spill` only frees memory if at + // least one record has been inserted in the in-memory sorter. This will not be the case if + // we have spilled in the previous step. + // + // If we did not spill in the previous step then `growPointerArrayIfNecessary` will be a + // no-op that does not allocate any memory, and therefore can't cause a spill event. + // + // Thus there is no need to call `acquireNewPageIfNecessary` again after this step. + growPointerArrayIfNecessary(); + } + /** * Write a record to the sorter. */ @@ -398,11 +446,10 @@ public void insertRecord( spill(); } - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); // Need 4 or 8 bytes to store the record length. final int required = length + uaoSize; - acquireNewPageIfNecessary(required); + allocateMemoryForRecordIfNecessary(required); final Object base = currentPage.getBaseObject(); final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); @@ -425,10 +472,9 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull) throws IOException { - growPointerArrayIfNecessary(); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); final int required = keyLen + valueLen + (2 * uaoSize); - acquireNewPageIfNecessary(required); + allocateMemoryForRecordIfNecessary(required); final Object base = currentPage.getBaseObject(); final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); @@ -572,7 +618,7 @@ public long spill() throws IOException { assert(inMemSorter != null); released += inMemSorter.getMemoryUsage(); totalSortTimeNanos += inMemSorter.getSortTimeNanos(); - inMemSorter.free(); + inMemSorter.freeMemory(); inMemSorter = null; taskContext.taskMetrics().incMemoryBytesSpilled(released); taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten()); @@ -669,7 +715,7 @@ public UnsafeSorterIterator getIterator(int startIndex) throws IOException { } i += spillWriter.recordsSpilled(); } - if (inMemSorter != null) { + if (inMemSorter != null && inMemSorter.numRecords() > 0) { UnsafeSorterIterator iter = inMemSorter.getSortedIterator(); moveOver(iter, startIndex - i); queue.add(iter); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index ff641a24a7b3e..33be899b6b438 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -159,32 +159,26 @@ private int getUsableCapacity() { return (int) (array.size() / (radixSortSupport != null ? 2 : 1.5)); } + public long getInitialSize() { + return initialSize; + } + /** * Free the memory used by pointer array. */ - public void free() { + public void freeMemory() { if (consumer != null) { if (array != null) { consumer.freeArray(array); } - array = null; - } - } - public void reset() { - if (consumer != null) { - consumer.freeArray(array); - // the call to consumer.allocateArray may trigger a spill which in turn access this instance - // and eventually re-enter this method and try to free the array again. by setting the array - // to null and its length to 0 we effectively make the spill code-path a no-op. setting the - // array to null also indicates that it has already been de-allocated which prevents a double - // de-allocation in free(). + // Set the array to null instead of allocating a new array. Allocating an array could have + // triggered another spill and this method already is called from UnsafeExternalSorter when + // spilling. Attempting to allocate while spilling is dangerous, as we could be holding onto + // a large partially complete allocation, which may prevent other memory from being allocated. + // Instead we will allocate the new array when it is necessary. array = null; usableCapacity = 0; - pos = 0; - nullBoundaryPos = 0; - array = consumer.allocateArray(initialSize); - usableCapacity = getUsableCapacity(); } pos = 0; nullBoundaryPos = 0; @@ -217,18 +211,20 @@ public boolean hasSpaceForAnotherRecord() { } public void expandPointerArray(LongArray newArray) { - if (newArray.size() < array.size()) { - // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); - // checkstyle.on: RegexpSinglelineJava + if (array != null) { + if (newArray.size() < array.size()) { + // checkstyle.off: RegexpSinglelineJava + throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); + // checkstyle.on: RegexpSinglelineJava + } + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L); + consumer.freeArray(array); } - Platform.copyMemory( - array.getBaseObject(), - array.getBaseOffset(), - newArray.getBaseObject(), - newArray.getBaseOffset(), - pos * 8L); - consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); } @@ -347,6 +343,11 @@ public long getCurrentPageNumber() { * {@code next()} will return the same mutable object. */ public UnsafeSorterIterator getSortedIterator() { + if (numRecords() == 0) { + // `array` might be null, so make sure that it is not accessed by returning early. + return new SortedIterator(0, 0); + } + int offset = 0; long start = System.nanoTime(); if (sortComparator != null) { diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index a1b66ccfaef03..dc2b4814c8284 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -23,7 +23,6 @@ import java.util.LinkedList; import java.util.UUID; -import org.hamcrest.Matchers; import scala.Tuple2$; import org.junit.After; @@ -38,7 +37,6 @@ import org.apache.spark.executor.TaskMetrics; import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.JavaSerializer; import org.apache.spark.serializer.SerializerInstance; @@ -581,40 +579,28 @@ public void testGetIterator() throws Exception { } @Test - public void testOOMDuringSpill() throws Exception { + public void testNoOOMDuringSpill() throws Exception { final UnsafeExternalSorter sorter = newSorter(); - // we assume that given default configuration, - // the size of the data we insert to the sorter (ints) - // and assuming we shouldn't spill before pointers array is exhausted - // (memory manager is not configured to throw at this point) - // - so this loop runs a reasonable number of iterations (<2000). - // test indeed completed within <30ms (on a quad i7 laptop). - for (int i = 0; sorter.hasSpaceForAnotherRecord(); ++i) { + for (int i = 0; i < 100; i++) { insertNumber(sorter, i); } - // we expect the next insert to attempt growing the pointerssArray first - // allocation is expected to fail, then a spill is triggered which - // attempts another allocation which also fails and we expect to see this - // OOM here. the original code messed with a released array within the - // spill code and ended up with a failed assertion. we also expect the - // location of the OOM to be - // org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset - memoryManager.markconsequentOOM(2); - try { - insertNumber(sorter, 1024); - fail("expected OutOfMmoryError but it seems operation surprisingly succeeded"); - } - // we expect an SparkOutOfMemoryError here, anything else (i.e the original NPE is a failure) - catch (SparkOutOfMemoryError oom){ - String oomStackTrace = Utils.exceptionString(oom); - assertThat("expected SparkOutOfMemoryError in " + - "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset", - oomStackTrace, - Matchers.containsString( - "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset")); + + // Check that spilling still succeeds when the task is starved for memory. + memoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.spill(); + memoryManager.resetConsequentOOM(); + + // Ensure that records can be appended after spilling, i.e. check that the sorter will allocate + // the new pointer array that it could not allocate while spilling. + for (int i = 0; i < 100; ++i) { + insertNumber(sorter, i); } + + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); } + private void verifyIntIterator(UnsafeSorterIterator iter, int start, int end) throws IOException { for (int i = start; i < end; i++) { diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 2b8a0602730e1..9d4909ddce792 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -20,6 +20,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; +import org.apache.spark.unsafe.array.LongArray; import org.junit.Assert; import org.junit.Test; @@ -27,7 +28,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -37,7 +37,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.isIn; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; public class UnsafeInMemorySorterSuite { @@ -147,7 +146,7 @@ public int compare( } @Test - public void freeAfterOOM() { + public void testNoOOMDuringReset() { final SparkConf sparkConf = new SparkConf(); sparkConf.set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); @@ -156,12 +155,7 @@ public void freeAfterOOM() { final TaskMemoryManager memoryManager = new TaskMemoryManager( testMemoryManager, 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); - final Object baseObject = dataPage.getBaseObject(); - // Write the records into the data page: - long position = dataPage.getBaseOffset(); - final HashPartitioner hashPartitioner = new HashPartitioner(4); // Use integer comparison for comparing prefixes (which are partition ids, in this case) final PrefixComparator prefixComparator = PrefixComparators.LONG; final RecordComparator recordComparator = new RecordComparator() { @@ -179,18 +173,24 @@ public int compare( UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, recordComparator, prefixComparator, 100, shouldUseRadixSort()); - testMemoryManager.markExecutionAsOutOfMemoryOnce(); - try { - sorter.reset(); - fail("expected SparkOutOfMemoryError but it seems operation surprisingly succeeded"); - } catch (SparkOutOfMemoryError oom) { - // as expected - } - // [SPARK-21907] this failed on NPE at - // org.apache.spark.memory.MemoryConsumer.freeArray(MemoryConsumer.java:108) - sorter.free(); - // simulate a 'back to back' free. - sorter.free(); + // Ensure that the sorter does not OOM while freeing its memory. + testMemoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.freeMemory(); + testMemoryManager.resetConsequentOOM(); + Assert.assertFalse(sorter.hasSpaceForAnotherRecord()); + + // Get the sorter in an usable state again by allocating a new pointer array. + LongArray array = consumer.allocateArray(1000); + sorter.expandPointerArray(array); + + // Ensure that it is safe to call freeMemory() multiple times. + testMemoryManager.markconsequentOOM(Integer.MAX_VALUE); + sorter.freeMemory(); + sorter.freeMemory(); + testMemoryManager.resetConsequentOOM(); + Assert.assertFalse(sorter.hasSpaceForAnotherRecord()); + + assertEquals(0L, memoryManager.cleanUpAllAllocatedMemory()); } } diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 60f67699f81be..987f383c9c4fa 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -119,6 +119,14 @@ class TestMemoryManager(conf: SparkConf) consequentOOM += n } + /** + * Undos the effects of [[markExecutionAsOutOfMemoryOnce]] and [[markconsequentOOM]] and lets + * calls to [[acquireExecutionMemory()]] (if there is enough memory available). + */ + def resetConsequentOOM(): Unit = synchronized { + consequentOOM = 0 + } + def limit(avail: Long): Unit = synchronized { require(avail >= 0) available = avail From 7766fd13c9e7cb72b97fdfee224d3958fbe882a0 Mon Sep 17 00:00:00 2001 From: Akshat Bordia Date: Tue, 29 Sep 2020 08:38:43 -0500 Subject: [PATCH 02/59] [MINOR][DOCS] Fixing log message for better clarity Fixing log message for better clarity. Closes #29870 from akshatb1/master. Lead-authored-by: Akshat Bordia Co-authored-by: Akshat Bordia Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index dbd89d646ae54..427e98e616515 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -568,7 +568,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + - s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be no less than the value of " + + s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be greater than the value of " + s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") } From 711d8dd28afd9af92b025f9908534e5f1d575042 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 29 Sep 2020 16:46:04 +0000 Subject: [PATCH 03/59] [SPARK-33018][SQL] Fix estimate statistics issue if child has 0 bytes ### What changes were proposed in this pull request? This pr fix estimate statistics issue if child has 0 bytes. ### Why are the changes needed? The `sizeInBytes` can be `0` when AQE and CBO are enabled(`spark.sql.adaptive.enabled`=true, `spark.sql.cbo.enabled`=true and `spark.sql.cbo.planStats.enabled`=true). This will generate incorrect BroadcastJoin, resulting in Driver OOM. For example: ![SPARK-33018](https://user-images.githubusercontent.com/5399861/94457606-647e3d00-01e7-11eb-85ee-812ae6efe7bb.jpg) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test. Closes #29894 from wangyum/SPARK-33018. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../SizeInBytesOnlyStatsPlanVisitor.scala | 3 ++- .../statsEstimation/JoinEstimationSuite.scala | 22 +++++++++++++++++++ .../StatsEstimationTestBase.scala | 9 +++++--- 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala index da36db7ae1f5f..a586988fd3253 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -53,7 +53,8 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { */ override def default(p: LogicalPlan): Statistics = p match { case p: LeafNode => p.computeStats() - case _: LogicalPlan => Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).product) + case _: LogicalPlan => + Statistics(sizeInBytes = p.children.map(_.stats.sizeInBytes).filter(_ > 0L).product) } override def visitAggregate(p: Aggregate): Statistics = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala index 6c5a2b247fc23..cdfc863cc0212 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala @@ -551,4 +551,26 @@ class JoinEstimationSuite extends StatsEstimationTestBase { attributeStats = AttributeMap(Nil)) assert(join.stats == expectedStats) } + + test("SPARK-33018 Fix estimate statistics issue if child has 0 bytes") { + case class MyStatsTestPlan( + outputList: Seq[Attribute], + sizeInBytes: BigInt) extends LeafNode { + override def output: Seq[Attribute] = outputList + override def computeStats(): Statistics = Statistics(sizeInBytes = sizeInBytes) + } + + val left = MyStatsTestPlan( + outputList = Seq("key-1-2", "key-2-4").map(nameToAttr), + sizeInBytes = BigInt(100)) + + val right = MyStatsTestPlan( + outputList = Seq("key-1-2", "key-2-3").map(nameToAttr), + sizeInBytes = BigInt(0)) + + val join = Join(left, right, LeftOuter, + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) + + assert(join.stats == Statistics(sizeInBytes = 100)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala index 9dceca59f5b87..0a27e31b3c9f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala @@ -26,17 +26,20 @@ import org.apache.spark.sql.types.{IntegerType, StringType} trait StatsEstimationTestBase extends SparkFunSuite { - var originalValue: Boolean = false + var originalCBOValue: Boolean = false + var originalPlanStatsValue: Boolean = false override def beforeAll(): Unit = { super.beforeAll() // Enable stats estimation based on CBO. - originalValue = SQLConf.get.getConf(SQLConf.CBO_ENABLED) + originalCBOValue = SQLConf.get.getConf(SQLConf.CBO_ENABLED) + originalPlanStatsValue = SQLConf.get.getConf(SQLConf.PLAN_STATS_ENABLED) SQLConf.get.setConf(SQLConf.CBO_ENABLED, true) } override def afterAll(): Unit = { - SQLConf.get.setConf(SQLConf.CBO_ENABLED, originalValue) + SQLConf.get.setConf(SQLConf.CBO_ENABLED, originalCBOValue) + SQLConf.get.setConf(SQLConf.PLAN_STATS_ENABLED, originalPlanStatsValue) super.afterAll() } From cc06266ade5a4eb35089501a3b32736624208d4c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 29 Sep 2020 12:02:45 -0700 Subject: [PATCH 04/59] [SPARK-33019][CORE] Use spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=1 by default ### What changes were proposed in this pull request? Apache Spark 3.1's default Hadoop profile is `hadoop-3.2`. Instead of having a warning documentation, this PR aims to use a consistent and safer version of Apache Hadoop file output committer algorithm which is `v1`. This will prevent a silent correctness regression during migration from Apache Spark 2.4/3.0 to Apache Spark 3.1.0. Of course, if there is a user-provided configuration, `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2`, that will be used still. ### Why are the changes needed? Apache Spark provides multiple distributions with Hadoop 2.7 and Hadoop 3.2. `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version` depends on the Hadoop version. Apache Hadoop 3.0 switches the default algorithm from `v1` to `v2` and now there exists a discussion to remove `v2`. We had better provide a consistent default behavior of `v1` across various Spark distributions. - [MAPREDUCE-7282](https://issues.apache.org/jira/browse/MAPREDUCE-7282) MR v2 commit algorithm should be deprecated and not the default ### Does this PR introduce _any_ user-facing change? Yes. This changes the default behavior. Users can override this conf. ### How was this patch tested? Manual. **BEFORE (spark-3.0.1-bin-hadoop3.2)** ```scala scala> sc.version res0: String = 3.0.1 scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version") res1: String = 2 ``` **AFTER** ```scala scala> sc.hadoopConfiguration.get("mapreduce.fileoutputcommitter.algorithm.version") res0: String = 1 ``` Closes #29895 from dongjoon-hyun/SPARK-DEFAUT-COMMITTER. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 3 +++ docs/configuration.md | 10 ++-------- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 1180501e8c738..6f799a542bc1e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -462,6 +462,9 @@ private[spark] object SparkHadoopUtil { for ((key, value) <- conf.getAll if key.startsWith("spark.hadoop.")) { hadoopConf.set(key.substring("spark.hadoop.".length), value) } + if (conf.getOption("spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version").isEmpty) { + hadoopConf.set("mapreduce.fileoutputcommitter.algorithm.version", "1") + } } private def appendSparkHiveConfigs(conf: SparkConf, hadoopConf: Configuration): Unit = { diff --git a/docs/configuration.md b/docs/configuration.md index 8b6ae9d777cce..d825a589dfd31 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1761,16 +1761,10 @@ Apart from these, the following properties are also available, and may be useful spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version - Dependent on environment + 1 The file output committer algorithm version, valid algorithm version number: 1 or 2. - Version 2 may have better performance, but version 1 may handle failures better in certain situations, - as per MAPREDUCE-4815. - The default value depends on the Hadoop version used in an environment: - 1 for Hadoop versions lower than 3.0 - 2 for Hadoop versions 3.0 and higher - It's important to note that this can change back to 1 again in the future once MAPREDUCE-7282 - is fixed and merged. + Note that 2 may cause a correctness issue like MAPREDUCE-7282. 2.2.0 From 3a299aa6480ac22501512cd0310d31a441d7dfdc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 30 Sep 2020 21:37:29 +0900 Subject: [PATCH 05/59] [SPARK-32741][SQL] Check if the same ExprId refers to the unique attribute in logical plans ### What changes were proposed in this pull request? Some plan transformations (e.g., `RemoveNoopOperators`) implicitly assume the same `ExprId` refers to the unique attribute. But, `RuleExecutor` does not check this integrity between logical plan transformations. So, this PR intends to add this check in `isPlanIntegral` of `Analyzer`/`Optimizer`. This PR comes from the talk with cloud-fan viirya in https://github.com/apache/spark/pull/29485#discussion_r475346278 ### Why are the changes needed? For better logical plan integrity checking. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29585 from maropu/PlanIntegrityTest. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 11 ++- .../sql/catalyst/optimizer/Optimizer.scala | 15 ++-- .../sql/catalyst/optimizer/subquery.scala | 51 ++++++++------ .../catalyst/plans/logical/LogicalPlan.scala | 70 +++++++++++++++++++ .../optimizer/FoldablePropagationSuite.scala | 4 +- .../logical/LogicalPlanIntegritySuite.scala | 51 ++++++++++++++ .../sql/execution/adaptive/AQEOptimizer.scala | 8 ++- .../spark/sql/streaming/StreamSuite.scala | 7 +- 8 files changed, 181 insertions(+), 36 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6e1f371b1a2b5..77a6631b250e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -48,6 +48,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PartitionOverwriteMode, StoreAssignmentPolicy} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils /** * A trivial [[Analyzer]] with a dummy [[SessionCatalog]] and [[EmptyFunctionRegistry]]. @@ -136,6 +137,10 @@ class Analyzer( private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog + override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { + !Utils.isTesting || LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan) + } + override def isView(nameParts: Seq[String]): Boolean = v1SessionCatalog.isView(nameParts) // Only for tests. @@ -2777,8 +2782,8 @@ class Analyzer( // a resolved Aggregate will not have Window Functions. case f @ UnresolvedHaving(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) if child.resolved && - hasWindowFunction(aggregateExprs) && - a.expressions.forall(_.resolved) => + hasWindowFunction(aggregateExprs) && + a.expressions.forall(_.resolved) => val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) // Create an Aggregate operator to evaluate aggregation functions. val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) @@ -2795,7 +2800,7 @@ class Analyzer( // Aggregate without Having clause. case a @ Aggregate(groupingExprs, aggregateExprs, child) if hasWindowFunction(aggregateExprs) && - a.expressions.forall(_.resolved) => + a.expressions.forall(_.resolved) => val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) // Create an Aggregate operator to evaluate aggregation functions. val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 94970740d8d91..f2360150e47b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ @@ -44,9 +43,11 @@ abstract class Optimizer(catalogManager: CatalogManager) // Currently we check after the execution of each rule if a plan: // - is still resolved // - only host special expressions in supported operators + // - has globally-unique attribute IDs override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { !Utils.isTesting || (plan.resolved && - plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty) + plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && + LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan)) } override protected val excludedOnceBatches: Set[String] = @@ -1585,14 +1586,14 @@ object ReplaceDistinctWithAggregate extends Rule[LogicalPlan] { * Replaces logical [[Deduplicate]] operator with an [[Aggregate]] operator. */ object ReplaceDeduplicateWithAggregate extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Deduplicate(keys, child) if !child.isStreaming => + def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput { + case d @ Deduplicate(keys, child) if !child.isStreaming => val keyExprIds = keys.map(_.exprId) val aggCols = child.output.map { attr => if (keyExprIds.contains(attr.exprId)) { attr } else { - Alias(new First(attr).toAggregateExpression(), attr.name)(attr.exprId) + Alias(new First(attr).toAggregateExpression(), attr.name)() } } // SPARK-22951: Physical aggregate operators distinguishes global aggregation and grouping @@ -1601,7 +1602,9 @@ object ReplaceDeduplicateWithAggregate extends Rule[LogicalPlan] { // we append a literal when the grouping key list is empty so that the result aggregate // operator is properly treated as a grouping aggregation. val nonemptyKeys = if (keys.isEmpty) Literal(1) :: Nil else keys - Aggregate(nonemptyKeys, aggCols, child) + val newAgg = Aggregate(nonemptyKeys, aggCols, child) + val attrMapping = d.output.zip(newAgg.output) + newAgg -> attrMapping } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 7b696912aa465..a168dcd7a83f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -338,15 +338,20 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { /** * Extract all correlated scalar subqueries from an expression. The subqueries are collected using - * the given collector. The expression is rewritten and returned. + * the given collector. To avoid the reuse of `exprId`s, this method generates new `exprId` + * for the subqueries and rewrite references in the given `expression`. + * This method returns extracted subqueries and the corresponding `exprId`s and these values + * will be used later in `constructLeftJoins` for building the child plan that + * returns subquery output with the `exprId`s. */ private def extractCorrelatedScalarSubqueries[E <: Expression]( expression: E, - subqueries: ArrayBuffer[ScalarSubquery]): E = { + subqueries: ArrayBuffer[(ScalarSubquery, ExprId)]): E = { val newExpression = expression transform { case s: ScalarSubquery if s.children.nonEmpty => - subqueries += s - s.plan.output.head + val newExprId = NamedExpression.newExprId + subqueries += s -> newExprId + s.plan.output.head.withExprId(newExprId) } newExpression.asInstanceOf[E] } @@ -510,16 +515,16 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { */ private def constructLeftJoins( child: LogicalPlan, - subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { + subqueries: ArrayBuffer[(ScalarSubquery, ExprId)]): LogicalPlan = { subqueries.foldLeft(child) { - case (currentChild, ScalarSubquery(query, conditions, _)) => + case (currentChild, (ScalarSubquery(query, conditions, _), newExprId)) => val origOutput = query.output.head val resultWithZeroTups = evalSubqueryOnZeroTups(query) if (resultWithZeroTups.isEmpty) { // CASE 1: Subquery guaranteed not to have the COUNT bug Project( - currentChild.output :+ origOutput, + currentChild.output :+ Alias(origOutput, origOutput.name)(exprId = newExprId), Join(currentChild, query, LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // Subquery might have the COUNT bug. Add appropriate corrections. @@ -544,7 +549,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { Alias( If(IsNull(alwaysTrueRef), resultWithZeroTups.get, - aggValRef), origOutput.name)(exprId = origOutput.exprId), + aggValRef), origOutput.name)(exprId = newExprId), Join(currentChild, Project(query.output :+ alwaysTrueExpr, query), LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) @@ -571,7 +576,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { (IsNull(alwaysTrueRef), resultWithZeroTups.get), (Not(havingNode.get.condition), Literal.create(null, aggValRef.dataType))), aggValRef), - origOutput.name)(exprId = origOutput.exprId) + origOutput.name)(exprId = newExprId) Project( currentChild.output :+ caseExpr, @@ -588,36 +593,42 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { * Rewrite [[Filter]], [[Project]] and [[Aggregate]] plans containing correlated scalar * subqueries. */ - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput { case a @ Aggregate(grouping, expressions, child) => - val subqueries = ArrayBuffer.empty[ScalarSubquery] + val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) if (subqueries.nonEmpty) { // We currently only allow correlated subqueries in an aggregate if they are part of the // grouping expressions. As a result we need to replace all the scalar subqueries in the // grouping expressions by their result. val newGrouping = grouping.map { e => - subqueries.find(_.semanticEquals(e)).map(_.plan.output.head).getOrElse(e) + subqueries.find(_._1.semanticEquals(e)).map(_._1.plan.output.head).getOrElse(e) } - Aggregate(newGrouping, newExpressions, constructLeftJoins(child, subqueries)) + val newAgg = Aggregate(newGrouping, newExpressions, constructLeftJoins(child, subqueries)) + val attrMapping = a.output.zip(newAgg.output) + newAgg -> attrMapping } else { - a + a -> Nil } case p @ Project(expressions, child) => - val subqueries = ArrayBuffer.empty[ScalarSubquery] + val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) if (subqueries.nonEmpty) { - Project(newExpressions, constructLeftJoins(child, subqueries)) + val newProj = Project(newExpressions, constructLeftJoins(child, subqueries)) + val attrMapping = p.output.zip(newProj.output) + newProj -> attrMapping } else { - p + p -> Nil } case f @ Filter(condition, child) => - val subqueries = ArrayBuffer.empty[ScalarSubquery] + val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] val newCondition = extractCorrelatedScalarSubqueries(condition, subqueries) if (subqueries.nonEmpty) { - Project(f.output, Filter(newCondition, constructLeftJoins(child, subqueries))) + val newProj = Project(f.output, Filter(newCondition, constructLeftJoins(child, subqueries))) + val attrMapping = f.output.zip(newProj.output) + newProj -> attrMapping } else { - f + f -> Nil } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 96c550616065a..48dfc5fd57e63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -203,3 +203,73 @@ abstract class BinaryNode extends LogicalPlan { abstract class OrderPreservingUnaryNode extends UnaryNode { override final def outputOrdering: Seq[SortOrder] = child.outputOrdering } + +object LogicalPlanIntegrity { + + private def canGetOutputAttrs(p: LogicalPlan): Boolean = { + p.resolved && !p.expressions.exists { e => + e.collectFirst { + // We cannot call `output` in plans with a `ScalarSubquery` expr having no column, + // so, we filter out them in advance. + case s: ScalarSubquery if s.plan.schema.fields.isEmpty => true + }.isDefined + } + } + + /** + * Since some logical plans (e.g., `Union`) can build `AttributeReference`s in their `output`, + * this method checks if the same `ExprId` refers to attributes having the same data type + * in plan output. + */ + def hasUniqueExprIdsForOutput(plan: LogicalPlan): Boolean = { + val exprIds = plan.collect { case p if canGetOutputAttrs(p) => + // NOTE: we still need to filter resolved expressions here because the output of + // some resolved logical plans can have unresolved references, + // e.g., outer references in `ExistenceJoin`. + p.output.filter(_.resolved).map { a => (a.exprId, a.dataType) } + }.flatten + + val ignoredExprIds = plan.collect { + // NOTE: `Union` currently reuses input `ExprId`s for output references, but we cannot + // simply modify the code for assigning new `ExprId`s in `Union#output` because + // the modification will make breaking changes (See SPARK-32741(#29585)). + // So, this check just ignores the `exprId`s of `Union` output. + case u: Union if u.resolved => u.output.map(_.exprId) + }.flatten.toSet + + val groupedDataTypesByExprId = exprIds.filterNot { case (exprId, _) => + ignoredExprIds.contains(exprId) + }.groupBy(_._1).values.map(_.distinct) + + groupedDataTypesByExprId.forall(_.length == 1) + } + + /** + * This method checks if reference `ExprId`s are not reused when assigning a new `ExprId`. + * For example, it returns false if plan transformers create an alias having the same `ExprId` + * with one of reference attributes, e.g., `a#1 + 1 AS a#1`. + */ + def checkIfSameExprIdNotReused(plan: LogicalPlan): Boolean = { + plan.collect { case p if p.resolved => + p.expressions.forall { + case a: Alias => + // Even if a plan is resolved, `a.references` can return unresolved references, + // e.g., in `Grouping`/`GroupingID`, so we need to filter out them and + // check if the same `exprId` in `Alias` does not exist + // among reference `exprId`s. + !a.references.filter(_.resolved).map(_.exprId).exists(_ == a.exprId) + case _ => + true + } + }.forall(identity) + } + + /** + * This method checks if the same `ExprId` refers to an unique attribute in a plan tree. + * Some plan transformers (e.g., `RemoveNoopOperators`) rewrite logical + * plans based on this assumption. + */ + def checkIfExprIdsAreGloballyUnique(plan: LogicalPlan): Boolean = { + checkIfSameExprIdNotReused(plan) && hasUniqueExprIdsForOutput(plan) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala index fe43e8e288673..92e4fa345e2ad 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala @@ -156,8 +156,8 @@ class FoldablePropagationSuite extends PlanTest { val query = expand.where(a1.isNotNull).select(a1, a2).analyze val optimized = Optimize.execute(query) val correctExpand = expand.copy(projections = Seq( - Seq(Literal(null), c2), - Seq(c1, Literal(null)))) + Seq(Literal(null), Literal(2)), + Seq(Literal(1), Literal(null)))) val correctAnswer = correctExpand.where(a1.isNotNull).select(a1, a2).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala new file mode 100644 index 0000000000000..6f342b8d94379 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala @@ -0,0 +1,51 @@ +/* + * 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.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.types.LongType + +class LogicalPlanIntegritySuite extends PlanTest { + import LogicalPlanIntegrity._ + + case class OutputTestPlan(child: LogicalPlan, output: Seq[Attribute]) extends UnaryNode { + override val analyzed = true + } + + test("Checks if the same `ExprId` refers to a semantically-equal attribute in a plan output") { + val t = LocalRelation('a.int, 'b.int) + assert(hasUniqueExprIdsForOutput(OutputTestPlan(t, t.output))) + assert(!hasUniqueExprIdsForOutput(OutputTestPlan(t, t.output.zipWithIndex.map { + case (a, i) => AttributeReference(s"c$i", LongType)(a.exprId) + }))) + } + + test("Checks if reference ExprIds are not reused when assigning a new ExprId") { + val t = LocalRelation('a.int, 'b.int) + val Seq(a, b) = t.output + assert(checkIfSameExprIdNotReused(t.select(Alias(a + 1, "a")()))) + assert(!checkIfSameExprIdNotReused(t.select(Alias(a + 1, "a")(exprId = a.exprId)))) + assert(checkIfSameExprIdNotReused(t.select(Alias(a + 1, "a")(exprId = b.exprId)))) + assert(checkIfSameExprIdNotReused(t.select(Alias(a + b, "ab")()))) + assert(!checkIfSameExprIdNotReused(t.select(Alias(a + b, "ab")(exprId = a.exprId)))) + assert(!checkIfSameExprIdNotReused(t.select(Alias(a + b, "ab")(exprId = b.exprId)))) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala index c82b264a600ef..0170f8b2f71c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.adaptive -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity, PlanHelper} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -54,4 +54,10 @@ class AQEOptimizer(conf: SQLConf) extends RuleExecutor[LogicalPlan] { } } } + + override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { + !Utils.isTesting || (plan.resolved && + plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty && + LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(plan)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 9f3ff1a6708e4..8797e5ad64149 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -36,7 +36,6 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} -import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_MILLIS import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand @@ -47,7 +46,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.streaming.util.{BlockOnStopSourceProvider, StreamManualClock} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} import org.apache.spark.util.Utils class StreamSuite extends StreamTest { @@ -1268,7 +1267,7 @@ class StreamSuite extends StreamTest { } abstract class FakeSource extends StreamSourceProvider { - private val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + private val fakeSchema = StructType(StructField("a", LongType) :: Nil) override def sourceSchema( spark: SQLContext, @@ -1290,7 +1289,7 @@ class FakeDefaultSource extends FakeSource { new Source { private var offset = -1L - override def schema: StructType = StructType(StructField("a", IntegerType) :: Nil) + override def schema: StructType = StructType(StructField("a", LongType) :: Nil) override def getOffset: Option[Offset] = { if (offset >= 10) { From ece8d8e22cf7e3924e44c16f58028c323dc54356 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 30 Sep 2020 09:27:57 -0700 Subject: [PATCH 06/59] [SPARK-33006][K8S][DOCS] Add dynamic PVC usage example into K8s doc ### What changes were proposed in this pull request? This updates K8s document to describe new dynamic PVC features. ### Why are the changes needed? This will help the user use the new features easily. ### Does this PR introduce _any_ user-facing change? Yes, but it's a doc updates. ### How was this patch tested? Manual. Screen Shot 2020-09-28 at 3 54 53 PM Screen Shot 2020-09-28 at 3 55 07 PM Closes #29897 from dongjoon-hyun/SPARK-33006. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index d0c6012e00aa6..e9c292d21fd47 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -307,7 +307,18 @@ And, the claim name of a `persistentVolumeClaim` with volume name `checkpointpvc spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=check-point-pvc-claim ``` -The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. +The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. + +For example, you can mount a dynamically-created persistent volume claim per executor by using `OnDemand` as a claim name and `storageClass` and `sizeLimit` options like the following. This is useful in case of [Dynamic Allocation](configuration.html#dynamic-allocation). +``` +spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.claimName=OnDemand +spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.storageClass=gp +spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.sizeLimit=500Gi +spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.path=/data +spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.readOnly=false +``` + +For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. ## Local Storage @@ -318,6 +329,15 @@ Spark supports using volumes to spill data during shuffles and other operations. --conf spark.kubernetes.driver.volumes.[VolumeType].spark-local-dir-[VolumeName].mount.readOnly=false ``` +Specifically, you can use persistent volume claims if the jobs require large shuffle and sorting operations in executors. + +``` +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.claimName=OnDemand +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.storageClass=gp +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.options.sizeLimit=500Gi +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data +spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false +``` If no volume is set as local storage, Spark uses temporary scratch space to spill data to disk during shuffles and other operations. When using Kubernetes as the resource manager the pods will be created with an [emptyDir](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir) volume mounted for each directory listed in `spark.local.dir` or the environment variable `SPARK_LOCAL_DIRS` . If no directories are explicitly specified then a default directory is created and configured appropriately. From 3bdbb5546d2517dda6f71613927cc1783c87f319 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Thu, 1 Oct 2020 08:15:53 +0900 Subject: [PATCH 07/59] [SPARK-31753][SQL][DOCS][FOLLOW-UP] Add missing keywords in the SQL docs ### What changes were proposed in this pull request? update sql-ref docs, the following key words will be added in this PR. CLUSTERED BY SORTED BY INTO num_buckets BUCKETS ### Why are the changes needed? let more users know the sql key words usage ### Does this PR introduce _any_ user-facing change? No ![image](https://user-images.githubusercontent.com/46367746/94428281-0a6b8080-01c3-11eb-9ff3-899f8da602ca.png) ![image](https://user-images.githubusercontent.com/46367746/94428285-0d667100-01c3-11eb-8a54-90e7641d917b.png) ![image](https://user-images.githubusercontent.com/46367746/94428288-0f303480-01c3-11eb-9e1d-023538aa6e2d.png) ### How was this patch tested? generate html test Closes #29883 from GuoPhilipse/add-sql-missing-keywords. Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Co-authored-by: GuoPhilipse Signed-off-by: Takeshi Yamamuro --- ...-ref-syntax-ddl-create-table-datasource.md | 7 +++- ...-ref-syntax-ddl-create-table-hiveformat.md | 32 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-create-table-datasource.md b/docs/sql-ref-syntax-ddl-create-table-datasource.md index d334447a91011..ba0516afbbfad 100644 --- a/docs/sql-ref-syntax-ddl-create-table-datasource.md +++ b/docs/sql-ref-syntax-ddl-create-table-datasource.md @@ -67,7 +67,12 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI * **SORTED BY** - Determines the order in which the data is stored in buckets. Default is Ascending order. + Specifies an ordering of bucket columns. Optionally, one can use ASC for an ascending order or DESC for a descending order after any column names in the SORTED BY clause. + If not specified, ASC is assumed by default. + +* **INTO num_buckets BUCKETS** + + Specifies buckets numbers, which is used in `CLUSTERED BY` clause. * **LOCATION** diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index 7bf847df98150..3a8c8d5b1160a 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -31,6 +31,9 @@ CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier [ COMMENT table_comment ] [ PARTITIONED BY ( col_name2[:] col_type2 [ COMMENT col_comment2 ], ... ) | ( col_name1, col_name2, ... ) ] + [ CLUSTERED BY ( col_name1, col_name2, ...) + [ SORTED BY ( col_name1 [ ASC | DESC ], col_name2 [ ASC | DESC ], ... ) ] + INTO num_buckets BUCKETS ] [ ROW FORMAT row_format ] [ STORED AS file_format ] [ LOCATION path ] @@ -65,6 +68,21 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI Partitions are created on the table, based on the columns specified. +* **CLUSTERED BY** + + Partitions created on the table will be bucketed into fixed buckets based on the column specified for bucketing. + + **NOTE:** Bucketing is an optimization technique that uses buckets (and bucketing columns) to determine data partitioning and avoid data shuffle. + +* **SORTED BY** + + Specifies an ordering of bucket columns. Optionally, one can use ASC for an ascending order or DESC for a descending order after any column names in the SORTED BY clause. + If not specified, ASC is assumed by default. + +* **INTO num_buckets BUCKETS** + + Specifies buckets numbers, which is used in `CLUSTERED BY` clause. + * **row_format** Use the `SERDE` clause to specify a custom SerDe for one table. Otherwise, use the `DELIMITED` clause to use the native SerDe and specify the delimiter, escape character, null character and so on. @@ -203,6 +221,20 @@ CREATE EXTERNAL TABLE family (id INT, name STRING) STORED AS INPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleInputFormat' OUTPUTFORMAT 'com.ly.spark.example.serde.io.SerDeExampleOutputFormat' LOCATION '/tmp/family/'; + +--Use `CLUSTERED BY` clause to create bucket table without `SORTED BY` +CREATE TABLE clustered_by_test1 (ID INT, AGE STRING) + CLUSTERED BY (ID) + INTO 4 BUCKETS + STORED AS ORC + +--Use `CLUSTERED BY` clause to create bucket table with `SORTED BY` +CREATE TABLE clustered_by_test2 (ID INT, NAME STRING) + PARTITIONED BY (YEAR STRING) + CLUSTERED BY (ID, NAME) + SORTED BY (ID ASC) + INTO 3 BUCKETS + STORED AS PARQUET ``` ### Related Statements From d75222dd1b0cdaaa7c22964e974117923fd069bb Mon Sep 17 00:00:00 2001 From: jlafleche Date: Wed, 30 Sep 2020 19:00:18 -0700 Subject: [PATCH 08/59] [SPARK-33012][BUILD][K8S] Upgrade fabric8 to 4.10.3 ### What changes were proposed in this pull request? This PR aims to upgrade `kubernetes-client` library to track fabric8's declared compatibility for k8s 1.18.0: https://github.com/fabric8io/kubernetes-client#compatibility-matrix ### Why are the changes needed? According to fabric8, 4.9.2 is incompatible with k8s 1.18.0. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Not tested yet. Closes #29888 from laflechejonathan/jlf/fabric8Ugprade. Authored-by: jlafleche Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 28 +++++++++++++++---- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 28 +++++++++++++++---- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 28 +++++++++++++++---- resource-managers/kubernetes/core/pom.xml | 2 +- .../kubernetes/integration-tests/pom.xml | 2 +- 5 files changed, 71 insertions(+), 17 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 900ee6d18d06d..fef1a6442cd33 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -139,14 +139,31 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/1.7.30//jul-to-slf4j-1.7.30.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar +kubernetes-client/4.10.3//kubernetes-client-4.10.3.jar +kubernetes-model-admissionregistration/4.10.3//kubernetes-model-admissionregistration-4.10.3.jar +kubernetes-model-apiextensions/4.10.3//kubernetes-model-apiextensions-4.10.3.jar +kubernetes-model-apps/4.10.3//kubernetes-model-apps-4.10.3.jar +kubernetes-model-autoscaling/4.10.3//kubernetes-model-autoscaling-4.10.3.jar +kubernetes-model-batch/4.10.3//kubernetes-model-batch-4.10.3.jar +kubernetes-model-certificates/4.10.3//kubernetes-model-certificates-4.10.3.jar +kubernetes-model-common/4.10.3//kubernetes-model-common-4.10.3.jar +kubernetes-model-coordination/4.10.3//kubernetes-model-coordination-4.10.3.jar +kubernetes-model-core/4.10.3//kubernetes-model-core-4.10.3.jar +kubernetes-model-discovery/4.10.3//kubernetes-model-discovery-4.10.3.jar +kubernetes-model-events/4.10.3//kubernetes-model-events-4.10.3.jar +kubernetes-model-extensions/4.10.3//kubernetes-model-extensions-4.10.3.jar +kubernetes-model-metrics/4.10.3//kubernetes-model-metrics-4.10.3.jar +kubernetes-model-networking/4.10.3//kubernetes-model-networking-4.10.3.jar +kubernetes-model-policy/4.10.3//kubernetes-model-policy-4.10.3.jar +kubernetes-model-rbac/4.10.3//kubernetes-model-rbac-4.10.3.jar +kubernetes-model-scheduling/4.10.3//kubernetes-model-scheduling-4.10.3.jar +kubernetes-model-settings/4.10.3//kubernetes-model-settings-4.10.3.jar +kubernetes-model-storageclass/4.10.3//kubernetes-model-storageclass-4.10.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar @@ -159,9 +176,10 @@ metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar -okhttp/3.12.6//okhttp-3.12.6.jar +okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar +openshift-model/4.10.3//openshift-model-4.10.3.jar orc-core/1.5.10/nohive/orc-core-1.5.10-nohive.jar orc-mapreduce/1.5.10/nohive/orc-mapreduce-1.5.10-nohive.jar orc-shims/1.5.10//orc-shims-1.5.10.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 9e167256236c0..6d1934b46261b 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -154,14 +154,31 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/1.7.30//jul-to-slf4j-1.7.30.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar +kubernetes-client/4.10.3//kubernetes-client-4.10.3.jar +kubernetes-model-admissionregistration/4.10.3//kubernetes-model-admissionregistration-4.10.3.jar +kubernetes-model-apiextensions/4.10.3//kubernetes-model-apiextensions-4.10.3.jar +kubernetes-model-apps/4.10.3//kubernetes-model-apps-4.10.3.jar +kubernetes-model-autoscaling/4.10.3//kubernetes-model-autoscaling-4.10.3.jar +kubernetes-model-batch/4.10.3//kubernetes-model-batch-4.10.3.jar +kubernetes-model-certificates/4.10.3//kubernetes-model-certificates-4.10.3.jar +kubernetes-model-common/4.10.3//kubernetes-model-common-4.10.3.jar +kubernetes-model-coordination/4.10.3//kubernetes-model-coordination-4.10.3.jar +kubernetes-model-core/4.10.3//kubernetes-model-core-4.10.3.jar +kubernetes-model-discovery/4.10.3//kubernetes-model-discovery-4.10.3.jar +kubernetes-model-events/4.10.3//kubernetes-model-events-4.10.3.jar +kubernetes-model-extensions/4.10.3//kubernetes-model-extensions-4.10.3.jar +kubernetes-model-metrics/4.10.3//kubernetes-model-metrics-4.10.3.jar +kubernetes-model-networking/4.10.3//kubernetes-model-networking-4.10.3.jar +kubernetes-model-policy/4.10.3//kubernetes-model-policy-4.10.3.jar +kubernetes-model-rbac/4.10.3//kubernetes-model-rbac-4.10.3.jar +kubernetes-model-scheduling/4.10.3//kubernetes-model-scheduling-4.10.3.jar +kubernetes-model-settings/4.10.3//kubernetes-model-settings-4.10.3.jar +kubernetes-model-storageclass/4.10.3//kubernetes-model-storageclass-4.10.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar @@ -174,9 +191,10 @@ metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar minlog/1.3.0//minlog-1.3.0.jar netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar objenesis/2.6//objenesis-2.6.jar -okhttp/3.12.6//okhttp-3.12.6.jar +okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar +openshift-model/4.10.3//openshift-model-4.10.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar orc-shims/1.5.10//orc-shims-1.5.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index b44b461014cd7..2e29d831b9e66 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -166,14 +166,31 @@ kerby-pkix/1.0.1//kerby-pkix-1.0.1.jar kerby-util/1.0.1//kerby-util-1.0.1.jar kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.9.2//kubernetes-client-4.9.2.jar -kubernetes-model-common/4.9.2//kubernetes-model-common-4.9.2.jar -kubernetes-model/4.9.2//kubernetes-model-4.9.2.jar +kubernetes-client/4.10.3//kubernetes-client-4.10.3.jar +kubernetes-model-admissionregistration/4.10.3//kubernetes-model-admissionregistration-4.10.3.jar +kubernetes-model-apiextensions/4.10.3//kubernetes-model-apiextensions-4.10.3.jar +kubernetes-model-apps/4.10.3//kubernetes-model-apps-4.10.3.jar +kubernetes-model-autoscaling/4.10.3//kubernetes-model-autoscaling-4.10.3.jar +kubernetes-model-batch/4.10.3//kubernetes-model-batch-4.10.3.jar +kubernetes-model-certificates/4.10.3//kubernetes-model-certificates-4.10.3.jar +kubernetes-model-common/4.10.3//kubernetes-model-common-4.10.3.jar +kubernetes-model-coordination/4.10.3//kubernetes-model-coordination-4.10.3.jar +kubernetes-model-core/4.10.3//kubernetes-model-core-4.10.3.jar +kubernetes-model-discovery/4.10.3//kubernetes-model-discovery-4.10.3.jar +kubernetes-model-events/4.10.3//kubernetes-model-events-4.10.3.jar +kubernetes-model-extensions/4.10.3//kubernetes-model-extensions-4.10.3.jar +kubernetes-model-metrics/4.10.3//kubernetes-model-metrics-4.10.3.jar +kubernetes-model-networking/4.10.3//kubernetes-model-networking-4.10.3.jar +kubernetes-model-policy/4.10.3//kubernetes-model-policy-4.10.3.jar +kubernetes-model-rbac/4.10.3//kubernetes-model-rbac-4.10.3.jar +kubernetes-model-scheduling/4.10.3//kubernetes-model-scheduling-4.10.3.jar +kubernetes-model-settings/4.10.3//kubernetes-model-settings-4.10.3.jar +kubernetes-model-storageclass/4.10.3//kubernetes-model-storageclass-4.10.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar -logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar @@ -188,9 +205,10 @@ netty-all/4.1.51.Final//netty-all-4.1.51.Final.jar nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar objenesis/2.6//objenesis-2.6.jar okhttp/2.7.5//okhttp-2.7.5.jar -okhttp/3.12.6//okhttp-3.12.6.jar +okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar +openshift-model/4.10.3//openshift-model-4.10.3.jar orc-core/1.5.10//orc-core-1.5.10.jar orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar orc-shims/1.5.10//orc-shims-1.5.10.jar diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index c1a7dafb69c46..a4c80f551cdfc 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -30,7 +30,7 @@ kubernetes - 4.9.2 + 4.10.3 diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 4a55ead38aae2..952081030f5f3 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -28,7 +28,7 @@ 1.3.0 - 4.9.2 + 4.10.3 kubernetes-integration-tests From 0b5a379c1fb87aa536ebe9433e501dcf4f80ea60 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 30 Sep 2020 19:24:50 -0700 Subject: [PATCH 09/59] [SPARK-33023][CORE] Judge path of Windows need add condition `Utils.isWindows` ### What changes were proposed in this pull request? according to https://github.com/apache/spark/pull/29881#discussion_r496648397 we need add condition `Utils.isWindows` ### Why are the changes needed? add strict condition of judging path is window path ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No Closes #29909 from AngersZhuuuu/SPARK-33023. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 409e3065492b0..501e865c4105a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1899,7 +1899,7 @@ class SparkContext(config: SparkConf) extends Logging { if (path == null || path.isEmpty) { logWarning("null or empty path specified as parameter to addJar") } else { - val key = if (path.contains("\\")) { + val key = if (path.contains("\\") && Utils.isWindows) { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { From 28ed3a512ac6fcaafa885eb8092a68fe9e8f5c26 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Wed, 30 Sep 2020 21:30:17 -0700 Subject: [PATCH 10/59] [SPARK-32723][WEBUI] Upgrade to jQuery 3.5.1 ### What changes were proposed in this pull request? Upgrade to the latest available version of jQuery (3.5.1). ### Why are the changes needed? There are some CVE-s reported (CVE-2020-11022, CVE-2020-11023) affecting older versions of jQuery. Although Spark UI is read-only and those CVEs doesn't seem to affect Spark, using the latest version of this library can help to handle vulnerability reports of security scans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual tests and checked the jQuery 3.5 upgrade guide. Closes #29902 from peter-toth/SPARK-32723-upgrade-to-jquery-3.5.1. Authored-by: Peter Toth Signed-off-by: Dongjoon Hyun --- .../resources/org/apache/spark/ui/static/jquery-3.4.1.min.js | 2 -- .../resources/org/apache/spark/ui/static/jquery-3.5.1.min.js | 2 ++ core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- dev/.rat-excludes | 2 +- docs/_layouts/global.html | 2 +- docs/js/vendor/jquery-3.4.1.min.js | 2 -- docs/js/vendor/jquery-3.5.1.min.js | 2 ++ 7 files changed, 7 insertions(+), 7 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/jquery-3.5.1.min.js delete mode 100644 docs/js/vendor/jquery-3.4.1.min.js create mode 100644 docs/js/vendor/jquery-3.5.1.min.js diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js deleted file mode 100644 index 07c00cd227da0..0000000000000 --- a/core/src/main/resources/org/apache/spark/ui/static/jquery-3.4.1.min.js +++ /dev/null @@ -1,2 +0,0 @@ -/*! jQuery v3.4.1 | (c) JS Foundation and other contributors | jquery.org/license */ -!function(e,t){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=e.document?t(e,!0):function(e){if(!e.document)throw new Error("jQuery requires a window with a document");return t(e)}:t(e)}("undefined"!=typeof window?window:this,function(C,e){"use strict";var t=[],E=C.document,r=Object.getPrototypeOf,s=t.slice,g=t.concat,u=t.push,i=t.indexOf,n={},o=n.toString,v=n.hasOwnProperty,a=v.toString,l=a.call(Object),y={},m=function(e){return"function"==typeof e&&"number"!=typeof e.nodeType},x=function(e){return null!=e&&e===e.window},c={type:!0,src:!0,nonce:!0,noModule:!0};function b(e,t,n){var r,i,o=(n=n||E).createElement("script");if(o.text=e,t)for(r in c)(i=t[r]||t.getAttribute&&t.getAttribute(r))&&o.setAttribute(r,i);n.head.appendChild(o).parentNode.removeChild(o)}function w(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?n[o.call(e)]||"object":typeof e}var f="3.4.1",k=function(e,t){return new k.fn.init(e,t)},p=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g;function d(e){var t=!!e&&"length"in e&&e.length,n=w(e);return!m(e)&&!x(e)&&("array"===n||0===t||"number"==typeof t&&0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp($),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+$),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),ne=function(e,t,n){var r="0x"+t-65536;return r!=r||n?t:r<0?String.fromCharCode(r+65536):String.fromCharCode(r>>10|55296,1023&r|56320)},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(m.childNodes),m.childNodes),t[m.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&((e?e.ownerDocument||e:m)!==C&&T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!A[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&U.test(t)){(s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=k),o=(l=h(t)).length;while(o--)l[o]="#"+s+" "+xe(l[o]);c=l.join(","),f=ee.test(t)&&ye(e.parentNode)||e}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){A(t,!0)}finally{s===k&&e.removeAttribute("id")}}}return g(t.replace(B,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[k]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:m;return r!==C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),m!==C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=k,!C.getElementsByName||!C.getElementsByName(k).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+k+"-]").length||v.push("~="),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+k+"+*").length||v.push(".#.+[+~]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",$)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)===(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e===C||e.ownerDocument===m&&y(m,e)?-1:t===C||t.ownerDocument===m&&y(m,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e===C?-1:t===C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]===m?-1:s[r]===m?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if((e.ownerDocument||e)!==C&&T(e),d.matchesSelector&&E&&!A[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){A(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=p[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&p(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function j(e,n,r){return m(n)?k.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?k.grep(e,function(e){return e===n!==r}):"string"!=typeof n?k.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(k.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||q,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:L.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof k?t[0]:t,k.merge(this,k.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),D.test(r[1])&&k.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(k):k.makeArray(e,this)}).prototype=k.fn,q=k(E);var H=/^(?:parents|prev(?:Until|All))/,O={children:!0,contents:!0,next:!0,prev:!0};function P(e,t){while((e=e[t])&&1!==e.nodeType);return e}k.fn.extend({has:function(e){var t=k(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i,ge={option:[1,""],thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?k.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;nx",y.noCloneChecked=!!me.cloneNode(!0).lastChild.defaultValue;var Te=/^key/,Ce=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,Ee=/^([^.]*)(?:\.(.+)|)/;function ke(){return!0}function Se(){return!1}function Ne(e,t){return e===function(){try{return E.activeElement}catch(e){}}()==("focus"===t)}function Ae(e,t,n,r,i,o){var a,s;if("object"==typeof t){for(s in"string"!=typeof n&&(r=r||n,n=void 0),t)Ae(e,s,n,r,t[s],o);return e}if(null==r&&null==i?(i=n,r=n=void 0):null==i&&("string"==typeof n?(i=r,r=void 0):(i=r,r=n,n=void 0)),!1===i)i=Se;else if(!i)return e;return 1===o&&(a=i,(i=function(e){return k().off(e),a.apply(this,arguments)}).guid=a.guid||(a.guid=k.guid++)),e.each(function(){k.event.add(this,t,i,r,n)})}function De(e,i,o){o?(Q.set(e,i,!1),k.event.add(e,i,{namespace:!1,handler:function(e){var t,n,r=Q.get(this,i);if(1&e.isTrigger&&this[i]){if(r.length)(k.event.special[i]||{}).delegateType&&e.stopPropagation();else if(r=s.call(arguments),Q.set(this,i,r),t=o(this,i),this[i](),r!==(n=Q.get(this,i))||t?Q.set(this,i,!1):n={},r!==n)return e.stopImmediatePropagation(),e.preventDefault(),n.value}else r.length&&(Q.set(this,i,{value:k.event.trigger(k.extend(r[0],k.Event.prototype),r.slice(1),this)}),e.stopImmediatePropagation())}})):void 0===Q.get(e,i)&&k.event.add(e,i,ke)}k.event={global:{},add:function(t,e,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.get(t);if(v){n.handler&&(n=(o=n).handler,i=o.selector),i&&k.find.matchesSelector(ie,i),n.guid||(n.guid=k.guid++),(u=v.events)||(u=v.events={}),(a=v.handle)||(a=v.handle=function(e){return"undefined"!=typeof k&&k.event.triggered!==e.type?k.event.dispatch.apply(t,arguments):void 0}),l=(e=(e||"").match(R)||[""]).length;while(l--)d=g=(s=Ee.exec(e[l])||[])[1],h=(s[2]||"").split(".").sort(),d&&(f=k.event.special[d]||{},d=(i?f.delegateType:f.bindType)||d,f=k.event.special[d]||{},c=k.extend({type:d,origType:g,data:r,handler:n,guid:n.guid,selector:i,needsContext:i&&k.expr.match.needsContext.test(i),namespace:h.join(".")},o),(p=u[d])||((p=u[d]=[]).delegateCount=0,f.setup&&!1!==f.setup.call(t,r,h,a)||t.addEventListener&&t.addEventListener(d,a)),f.add&&(f.add.call(t,c),c.handler.guid||(c.handler.guid=n.guid)),i?p.splice(p.delegateCount++,0,c):p.push(c),k.event.global[d]=!0)}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.hasData(e)&&Q.get(e);if(v&&(u=v.events)){l=(t=(t||"").match(R)||[""]).length;while(l--)if(d=g=(s=Ee.exec(t[l])||[])[1],h=(s[2]||"").split(".").sort(),d){f=k.event.special[d]||{},p=u[d=(r?f.delegateType:f.bindType)||d]||[],s=s[2]&&new RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),a=o=p.length;while(o--)c=p[o],!i&&g!==c.origType||n&&n.guid!==c.guid||s&&!s.test(c.namespace)||r&&r!==c.selector&&("**"!==r||!c.selector)||(p.splice(o,1),c.selector&&p.delegateCount--,f.remove&&f.remove.call(e,c));a&&!p.length&&(f.teardown&&!1!==f.teardown.call(e,h,v.handle)||k.removeEvent(e,d,v.handle),delete u[d])}else for(d in u)k.event.remove(e,d+t[l],n,r,!0);k.isEmptyObject(u)&&Q.remove(e,"handle events")}},dispatch:function(e){var t,n,r,i,o,a,s=k.event.fix(e),u=new Array(arguments.length),l=(Q.get(this,"events")||{})[s.type]||[],c=k.event.special[s.type]||{};for(u[0]=s,t=1;t\x20\t\r\n\f]*)[^>]*)\/>/gi,qe=/\s*$/g;function Oe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&k(e).children("tbody")[0]||e}function Pe(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function Re(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Me(e,t){var n,r,i,o,a,s,u,l;if(1===t.nodeType){if(Q.hasData(e)&&(o=Q.access(e),a=Q.set(t,o),l=o.events))for(i in delete a.handle,a.events={},l)for(n=0,r=l[i].length;n")},clone:function(e,t,n){var r,i,o,a,s,u,l,c=e.cloneNode(!0),f=oe(e);if(!(y.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||k.isXMLDoc(e)))for(a=ve(c),r=0,i=(o=ve(e)).length;r").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Vt,Gt=[],Yt=/(=)\?(?=&|$)|\?\?/;k.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Gt.pop()||k.expando+"_"+kt++;return this[e]=!0,e}}),k.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Yt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Yt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Yt,"$1"+r):!1!==e.jsonp&&(e.url+=(St.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||k.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?k(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Gt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Vt=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Vt.childNodes.length),k.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=D.exec(e))?[t.createElement(i[1])]:(i=we([e],t,o),o&&o.length&&k(o).remove(),k.merge([],i.childNodes)));var r,i,o},k.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(k.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},k.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){k.fn[t]=function(e){return this.on(t,e)}}),k.expr.pseudos.animated=function(t){return k.grep(k.timers,function(e){return t===e.elem}).length},k.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=k.css(e,"position"),c=k(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=k.css(e,"top"),u=k.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,k.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):c.css(f)}},k.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){k.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===k.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===k.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=k(e).offset()).top+=k.css(e,"borderTopWidth",!0),i.left+=k.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-k.css(r,"marginTop",!0),left:t.left-i.left-k.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===k.css(e,"position"))e=e.offsetParent;return e||ie})}}),k.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;k.fn[t]=function(e){return _(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),k.each(["top","left"],function(e,n){k.cssHooks[n]=ze(y.pixelPosition,function(e,t){if(t)return t=_e(e,n),$e.test(t)?k(e).position()[n]+"px":t})}),k.each({Height:"height",Width:"width"},function(a,s){k.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){k.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return _(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?k.css(e,t,i):k.style(e,t,n,i)},s,n?e:void 0,n)}})}),k.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){k.fn[n]=function(e,t){return 0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp(F),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+F),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\[\\da-fA-F]{1,6}"+M+"?|\\\\([^\\r\\n\\f])","g"),ne=function(e,t){var n="0x"+e.slice(1)-65536;return t||(n<0?String.fromCharCode(n+65536):String.fromCharCode(n>>10|55296,1023&n|56320))},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(p.childNodes),p.childNodes),t[p.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&(T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!N[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&(U.test(t)||z.test(t))){(f=ee.test(t)&&ye(e.parentNode)||e)===e&&d.scope||((s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=S)),o=(l=h(t)).length;while(o--)l[o]=(s?"#"+s:":scope")+" "+xe(l[o]);c=l.join(",")}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){N(t,!0)}finally{s===S&&e.removeAttribute("id")}}}return g(t.replace($,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[S]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:p;return r!=C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),p!=C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.scope=ce(function(e){return a.appendChild(e).appendChild(C.createElement("div")),"undefined"!=typeof e.querySelectorAll&&!e.querySelectorAll(":scope fieldset div").length}),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=S,!C.getElementsByName||!C.getElementsByName(S).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){var t;a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+S+"-]").length||v.push("~="),(t=C.createElement("input")).setAttribute("name",""),e.appendChild(t),e.querySelectorAll("[name='']").length||v.push("\\["+M+"*name"+M+"*="+M+"*(?:''|\"\")"),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+S+"+*").length||v.push(".#.+[+~]"),e.querySelectorAll("\\\f"),v.push("[\\r\\n\\f]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",F)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)==(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e==C||e.ownerDocument==p&&y(p,e)?-1:t==C||t.ownerDocument==p&&y(p,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e==C?-1:t==C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]==p?-1:s[r]==p?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if(T(e),d.matchesSelector&&E&&!N[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){N(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=m[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&m(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function D(e,n,r){return m(n)?S.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?S.grep(e,function(e){return e===n!==r}):"string"!=typeof n?S.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(S.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||j,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:q.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof S?t[0]:t,S.merge(this,S.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),N.test(r[1])&&S.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(S):S.makeArray(e,this)}).prototype=S.fn,j=S(E);var L=/^(?:parents|prev(?:Until|All))/,H={children:!0,contents:!0,next:!0,prev:!0};function O(e,t){while((e=e[t])&&1!==e.nodeType);return e}S.fn.extend({has:function(e){var t=S(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i;ce=E.createDocumentFragment().appendChild(E.createElement("div")),(fe=E.createElement("input")).setAttribute("type","radio"),fe.setAttribute("checked","checked"),fe.setAttribute("name","t"),ce.appendChild(fe),y.checkClone=ce.cloneNode(!0).cloneNode(!0).lastChild.checked,ce.innerHTML="",y.noCloneChecked=!!ce.cloneNode(!0).lastChild.defaultValue,ce.innerHTML="",y.option=!!ce.lastChild;var ge={thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?S.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;n",""]);var me=/<|&#?\w+;/;function xe(e,t,n,r,i){for(var o,a,s,u,l,c,f=t.createDocumentFragment(),p=[],d=0,h=e.length;d\s*$/g;function qe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&S(e).children("tbody")[0]||e}function Le(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function He(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Oe(e,t){var n,r,i,o,a,s;if(1===t.nodeType){if(Y.hasData(e)&&(s=Y.get(e).events))for(i in Y.remove(t,"handle events"),s)for(n=0,r=s[i].length;n").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Ut,Xt=[],Vt=/(=)\?(?=&|$)|\?\?/;S.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Xt.pop()||S.expando+"_"+Ct.guid++;return this[e]=!0,e}}),S.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Vt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Vt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Vt,"$1"+r):!1!==e.jsonp&&(e.url+=(Et.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||S.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?S(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Xt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Ut=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Ut.childNodes.length),S.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=N.exec(e))?[t.createElement(i[1])]:(i=xe([e],t,o),o&&o.length&&S(o).remove(),S.merge([],i.childNodes)));var r,i,o},S.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(S.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},S.expr.pseudos.animated=function(t){return S.grep(S.timers,function(e){return t===e.elem}).length},S.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=S.css(e,"position"),c=S(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=S.css(e,"top"),u=S.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,S.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):("number"==typeof f.top&&(f.top+="px"),"number"==typeof f.left&&(f.left+="px"),c.css(f))}},S.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){S.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===S.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===S.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=S(e).offset()).top+=S.css(e,"borderTopWidth",!0),i.left+=S.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-S.css(r,"marginTop",!0),left:t.left-i.left-S.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===S.css(e,"position"))e=e.offsetParent;return e||re})}}),S.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;S.fn[t]=function(e){return $(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),S.each(["top","left"],function(e,n){S.cssHooks[n]=$e(y.pixelPosition,function(e,t){if(t)return t=Be(e,n),Me.test(t)?S(e).position()[n]+"px":t})}),S.each({Height:"height",Width:"width"},function(a,s){S.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){S.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return $(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?S.css(e,t,i):S.style(e,t,n,i)},s,n?e:void 0,n)}})}),S.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){S.fn[t]=function(e){return this.on(t,e)}}),S.fn.extend({bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},hover:function(e,t){return this.mouseenter(e).mouseleave(t||e)}}),S.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){S.fn[n]=function(e,t){return 0 - + diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 98786437f7b1c..0e892a927906a 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -25,7 +25,7 @@ bootstrap.bundle.min.js bootstrap.bundle.min.js.map bootstrap.min.css bootstrap.min.css.map -jquery-3.4.1.min.js +jquery-3.5.1.min.js d3.min.js dagre-d3.min.js graphlib-dot.min.js diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d6548f0fa9534..5f6cd7c6b7f20 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -168,7 +168,7 @@

{{ page.title }}

- + diff --git a/docs/js/vendor/jquery-3.4.1.min.js b/docs/js/vendor/jquery-3.4.1.min.js deleted file mode 100644 index 07c00cd227da0..0000000000000 --- a/docs/js/vendor/jquery-3.4.1.min.js +++ /dev/null @@ -1,2 +0,0 @@ -/*! jQuery v3.4.1 | (c) JS Foundation and other contributors | jquery.org/license */ -!function(e,t){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=e.document?t(e,!0):function(e){if(!e.document)throw new Error("jQuery requires a window with a document");return t(e)}:t(e)}("undefined"!=typeof window?window:this,function(C,e){"use strict";var t=[],E=C.document,r=Object.getPrototypeOf,s=t.slice,g=t.concat,u=t.push,i=t.indexOf,n={},o=n.toString,v=n.hasOwnProperty,a=v.toString,l=a.call(Object),y={},m=function(e){return"function"==typeof e&&"number"!=typeof e.nodeType},x=function(e){return null!=e&&e===e.window},c={type:!0,src:!0,nonce:!0,noModule:!0};function b(e,t,n){var r,i,o=(n=n||E).createElement("script");if(o.text=e,t)for(r in c)(i=t[r]||t.getAttribute&&t.getAttribute(r))&&o.setAttribute(r,i);n.head.appendChild(o).parentNode.removeChild(o)}function w(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?n[o.call(e)]||"object":typeof e}var f="3.4.1",k=function(e,t){return new k.fn.init(e,t)},p=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g;function d(e){var t=!!e&&"length"in e&&e.length,n=w(e);return!m(e)&&!x(e)&&("array"===n||0===t||"number"==typeof t&&0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp($),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+$),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),ne=function(e,t,n){var r="0x"+t-65536;return r!=r||n?t:r<0?String.fromCharCode(r+65536):String.fromCharCode(r>>10|55296,1023&r|56320)},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(m.childNodes),m.childNodes),t[m.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&((e?e.ownerDocument||e:m)!==C&&T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!A[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&U.test(t)){(s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=k),o=(l=h(t)).length;while(o--)l[o]="#"+s+" "+xe(l[o]);c=l.join(","),f=ee.test(t)&&ye(e.parentNode)||e}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){A(t,!0)}finally{s===k&&e.removeAttribute("id")}}}return g(t.replace(B,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[k]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:m;return r!==C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),m!==C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=k,!C.getElementsByName||!C.getElementsByName(k).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+k+"-]").length||v.push("~="),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+k+"+*").length||v.push(".#.+[+~]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",$)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)===(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e===C||e.ownerDocument===m&&y(m,e)?-1:t===C||t.ownerDocument===m&&y(m,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e===C?-1:t===C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]===m?-1:s[r]===m?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if((e.ownerDocument||e)!==C&&T(e),d.matchesSelector&&E&&!A[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){A(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=p[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&p(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function j(e,n,r){return m(n)?k.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?k.grep(e,function(e){return e===n!==r}):"string"!=typeof n?k.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(k.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||q,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:L.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof k?t[0]:t,k.merge(this,k.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),D.test(r[1])&&k.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(k):k.makeArray(e,this)}).prototype=k.fn,q=k(E);var H=/^(?:parents|prev(?:Until|All))/,O={children:!0,contents:!0,next:!0,prev:!0};function P(e,t){while((e=e[t])&&1!==e.nodeType);return e}k.fn.extend({has:function(e){var t=k(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i,ge={option:[1,""],thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?k.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;nx",y.noCloneChecked=!!me.cloneNode(!0).lastChild.defaultValue;var Te=/^key/,Ce=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,Ee=/^([^.]*)(?:\.(.+)|)/;function ke(){return!0}function Se(){return!1}function Ne(e,t){return e===function(){try{return E.activeElement}catch(e){}}()==("focus"===t)}function Ae(e,t,n,r,i,o){var a,s;if("object"==typeof t){for(s in"string"!=typeof n&&(r=r||n,n=void 0),t)Ae(e,s,n,r,t[s],o);return e}if(null==r&&null==i?(i=n,r=n=void 0):null==i&&("string"==typeof n?(i=r,r=void 0):(i=r,r=n,n=void 0)),!1===i)i=Se;else if(!i)return e;return 1===o&&(a=i,(i=function(e){return k().off(e),a.apply(this,arguments)}).guid=a.guid||(a.guid=k.guid++)),e.each(function(){k.event.add(this,t,i,r,n)})}function De(e,i,o){o?(Q.set(e,i,!1),k.event.add(e,i,{namespace:!1,handler:function(e){var t,n,r=Q.get(this,i);if(1&e.isTrigger&&this[i]){if(r.length)(k.event.special[i]||{}).delegateType&&e.stopPropagation();else if(r=s.call(arguments),Q.set(this,i,r),t=o(this,i),this[i](),r!==(n=Q.get(this,i))||t?Q.set(this,i,!1):n={},r!==n)return e.stopImmediatePropagation(),e.preventDefault(),n.value}else r.length&&(Q.set(this,i,{value:k.event.trigger(k.extend(r[0],k.Event.prototype),r.slice(1),this)}),e.stopImmediatePropagation())}})):void 0===Q.get(e,i)&&k.event.add(e,i,ke)}k.event={global:{},add:function(t,e,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.get(t);if(v){n.handler&&(n=(o=n).handler,i=o.selector),i&&k.find.matchesSelector(ie,i),n.guid||(n.guid=k.guid++),(u=v.events)||(u=v.events={}),(a=v.handle)||(a=v.handle=function(e){return"undefined"!=typeof k&&k.event.triggered!==e.type?k.event.dispatch.apply(t,arguments):void 0}),l=(e=(e||"").match(R)||[""]).length;while(l--)d=g=(s=Ee.exec(e[l])||[])[1],h=(s[2]||"").split(".").sort(),d&&(f=k.event.special[d]||{},d=(i?f.delegateType:f.bindType)||d,f=k.event.special[d]||{},c=k.extend({type:d,origType:g,data:r,handler:n,guid:n.guid,selector:i,needsContext:i&&k.expr.match.needsContext.test(i),namespace:h.join(".")},o),(p=u[d])||((p=u[d]=[]).delegateCount=0,f.setup&&!1!==f.setup.call(t,r,h,a)||t.addEventListener&&t.addEventListener(d,a)),f.add&&(f.add.call(t,c),c.handler.guid||(c.handler.guid=n.guid)),i?p.splice(p.delegateCount++,0,c):p.push(c),k.event.global[d]=!0)}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.hasData(e)&&Q.get(e);if(v&&(u=v.events)){l=(t=(t||"").match(R)||[""]).length;while(l--)if(d=g=(s=Ee.exec(t[l])||[])[1],h=(s[2]||"").split(".").sort(),d){f=k.event.special[d]||{},p=u[d=(r?f.delegateType:f.bindType)||d]||[],s=s[2]&&new RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),a=o=p.length;while(o--)c=p[o],!i&&g!==c.origType||n&&n.guid!==c.guid||s&&!s.test(c.namespace)||r&&r!==c.selector&&("**"!==r||!c.selector)||(p.splice(o,1),c.selector&&p.delegateCount--,f.remove&&f.remove.call(e,c));a&&!p.length&&(f.teardown&&!1!==f.teardown.call(e,h,v.handle)||k.removeEvent(e,d,v.handle),delete u[d])}else for(d in u)k.event.remove(e,d+t[l],n,r,!0);k.isEmptyObject(u)&&Q.remove(e,"handle events")}},dispatch:function(e){var t,n,r,i,o,a,s=k.event.fix(e),u=new Array(arguments.length),l=(Q.get(this,"events")||{})[s.type]||[],c=k.event.special[s.type]||{};for(u[0]=s,t=1;t\x20\t\r\n\f]*)[^>]*)\/>/gi,qe=/\s*$/g;function Oe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&k(e).children("tbody")[0]||e}function Pe(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function Re(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Me(e,t){var n,r,i,o,a,s,u,l;if(1===t.nodeType){if(Q.hasData(e)&&(o=Q.access(e),a=Q.set(t,o),l=o.events))for(i in delete a.handle,a.events={},l)for(n=0,r=l[i].length;n")},clone:function(e,t,n){var r,i,o,a,s,u,l,c=e.cloneNode(!0),f=oe(e);if(!(y.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||k.isXMLDoc(e)))for(a=ve(c),r=0,i=(o=ve(e)).length;r").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Vt,Gt=[],Yt=/(=)\?(?=&|$)|\?\?/;k.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Gt.pop()||k.expando+"_"+kt++;return this[e]=!0,e}}),k.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Yt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Yt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Yt,"$1"+r):!1!==e.jsonp&&(e.url+=(St.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||k.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?k(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Gt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Vt=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Vt.childNodes.length),k.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=D.exec(e))?[t.createElement(i[1])]:(i=we([e],t,o),o&&o.length&&k(o).remove(),k.merge([],i.childNodes)));var r,i,o},k.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(k.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},k.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){k.fn[t]=function(e){return this.on(t,e)}}),k.expr.pseudos.animated=function(t){return k.grep(k.timers,function(e){return t===e.elem}).length},k.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=k.css(e,"position"),c=k(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=k.css(e,"top"),u=k.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,k.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):c.css(f)}},k.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){k.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===k.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===k.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=k(e).offset()).top+=k.css(e,"borderTopWidth",!0),i.left+=k.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-k.css(r,"marginTop",!0),left:t.left-i.left-k.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===k.css(e,"position"))e=e.offsetParent;return e||ie})}}),k.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;k.fn[t]=function(e){return _(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),k.each(["top","left"],function(e,n){k.cssHooks[n]=ze(y.pixelPosition,function(e,t){if(t)return t=_e(e,n),$e.test(t)?k(e).position()[n]+"px":t})}),k.each({Height:"height",Width:"width"},function(a,s){k.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){k.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return _(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?k.css(e,t,i):k.style(e,t,n,i)},s,n?e:void 0,n)}})}),k.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){k.fn[n]=function(e,t){return 0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp(F),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+F),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\[\\da-fA-F]{1,6}"+M+"?|\\\\([^\\r\\n\\f])","g"),ne=function(e,t){var n="0x"+e.slice(1)-65536;return t||(n<0?String.fromCharCode(n+65536):String.fromCharCode(n>>10|55296,1023&n|56320))},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(p.childNodes),p.childNodes),t[p.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&(T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!N[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&(U.test(t)||z.test(t))){(f=ee.test(t)&&ye(e.parentNode)||e)===e&&d.scope||((s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=S)),o=(l=h(t)).length;while(o--)l[o]=(s?"#"+s:":scope")+" "+xe(l[o]);c=l.join(",")}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){N(t,!0)}finally{s===S&&e.removeAttribute("id")}}}return g(t.replace($,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[S]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:p;return r!=C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),p!=C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.scope=ce(function(e){return a.appendChild(e).appendChild(C.createElement("div")),"undefined"!=typeof e.querySelectorAll&&!e.querySelectorAll(":scope fieldset div").length}),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=S,!C.getElementsByName||!C.getElementsByName(S).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){var t;a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+S+"-]").length||v.push("~="),(t=C.createElement("input")).setAttribute("name",""),e.appendChild(t),e.querySelectorAll("[name='']").length||v.push("\\["+M+"*name"+M+"*="+M+"*(?:''|\"\")"),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+S+"+*").length||v.push(".#.+[+~]"),e.querySelectorAll("\\\f"),v.push("[\\r\\n\\f]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",F)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)==(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e==C||e.ownerDocument==p&&y(p,e)?-1:t==C||t.ownerDocument==p&&y(p,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e==C?-1:t==C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]==p?-1:s[r]==p?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if(T(e),d.matchesSelector&&E&&!N[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){N(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=m[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&m(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function D(e,n,r){return m(n)?S.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?S.grep(e,function(e){return e===n!==r}):"string"!=typeof n?S.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(S.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||j,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:q.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof S?t[0]:t,S.merge(this,S.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),N.test(r[1])&&S.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(S):S.makeArray(e,this)}).prototype=S.fn,j=S(E);var L=/^(?:parents|prev(?:Until|All))/,H={children:!0,contents:!0,next:!0,prev:!0};function O(e,t){while((e=e[t])&&1!==e.nodeType);return e}S.fn.extend({has:function(e){var t=S(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i;ce=E.createDocumentFragment().appendChild(E.createElement("div")),(fe=E.createElement("input")).setAttribute("type","radio"),fe.setAttribute("checked","checked"),fe.setAttribute("name","t"),ce.appendChild(fe),y.checkClone=ce.cloneNode(!0).cloneNode(!0).lastChild.checked,ce.innerHTML="",y.noCloneChecked=!!ce.cloneNode(!0).lastChild.defaultValue,ce.innerHTML="",y.option=!!ce.lastChild;var ge={thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?S.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;n",""]);var me=/<|&#?\w+;/;function xe(e,t,n,r,i){for(var o,a,s,u,l,c,f=t.createDocumentFragment(),p=[],d=0,h=e.length;d\s*$/g;function qe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&S(e).children("tbody")[0]||e}function Le(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function He(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Oe(e,t){var n,r,i,o,a,s;if(1===t.nodeType){if(Y.hasData(e)&&(s=Y.get(e).events))for(i in Y.remove(t,"handle events"),s)for(n=0,r=s[i].length;n").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Ut,Xt=[],Vt=/(=)\?(?=&|$)|\?\?/;S.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Xt.pop()||S.expando+"_"+Ct.guid++;return this[e]=!0,e}}),S.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Vt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Vt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Vt,"$1"+r):!1!==e.jsonp&&(e.url+=(Et.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||S.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?S(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Xt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Ut=E.implementation.createHTMLDocument("").body).innerHTML="
",2===Ut.childNodes.length),S.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=N.exec(e))?[t.createElement(i[1])]:(i=xe([e],t,o),o&&o.length&&S(o).remove(),S.merge([],i.childNodes)));var r,i,o},S.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(S.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},S.expr.pseudos.animated=function(t){return S.grep(S.timers,function(e){return t===e.elem}).length},S.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=S.css(e,"position"),c=S(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=S.css(e,"top"),u=S.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,S.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):("number"==typeof f.top&&(f.top+="px"),"number"==typeof f.left&&(f.left+="px"),c.css(f))}},S.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){S.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===S.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===S.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=S(e).offset()).top+=S.css(e,"borderTopWidth",!0),i.left+=S.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-S.css(r,"marginTop",!0),left:t.left-i.left-S.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===S.css(e,"position"))e=e.offsetParent;return e||re})}}),S.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;S.fn[t]=function(e){return $(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),S.each(["top","left"],function(e,n){S.cssHooks[n]=$e(y.pixelPosition,function(e,t){if(t)return t=Be(e,n),Me.test(t)?S(e).position()[n]+"px":t})}),S.each({Height:"height",Width:"width"},function(a,s){S.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){S.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return $(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?S.css(e,t,i):S.style(e,t,n,i)},s,n?e:void 0,n)}})}),S.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){S.fn[t]=function(e){return this.on(t,e)}}),S.fn.extend({bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},hover:function(e,t){return this.mouseenter(e).mouseleave(t||e)}}),S.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){S.fn[n]=function(e,t){return 0 Date: Thu, 1 Oct 2020 14:50:32 +0900 Subject: [PATCH 11/59] [SPARK-32992][SQL] Map Oracle's ROWID type to StringType in read via JDBC ### What changes were proposed in this pull request? Convert the `ROWID` type in the Oracle JDBC dialect to Catalyst's `StringType`. The doc for Oracle 19c says explicitly that the type must be string: https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Data-Types.html#GUID-AEF1FE4C-2DE5-4BE7-BB53-83AD8F1E34EF ### Why are the changes needed? To avoid the exception showed in https://stackoverflow.com/questions/52244492/spark-jdbc-dataframereader-fails-to-read-oracle-table-with-datatype-as-rowid ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? N/A Closes #29884 from MaxGekk/jdbc-oracle-rowid-string. Authored-by: Max Gekk Signed-off-by: Takeshi Yamamuro --- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 11 +++++++++++ .../org/apache/spark/sql/jdbc/OracleDialect.scala | 6 ++++++ 2 files changed, 17 insertions(+) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 9c59023cd8766..ce63d1df6f028 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -518,4 +518,15 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSpark """.stripMargin.replaceAll("\n", " ")) assert(sql("select id, d, t from queryOption").collect.toSet == expectedResult) } + + test("SPARK-32992: map Oracle's ROWID type to StringType") { + val rows = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("query", "SELECT ROWID from datetime") + .load() + .collect() + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(0).equals("class java.lang.String")) + assert(!rows(0).getString(0).isEmpty) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 4c0623729e00d..3f12b9acd0fc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -64,6 +64,12 @@ private case object OracleDialect extends JdbcDialect { => Some(TimestampType) // Value for Timestamp with Time Zone in Oracle case BINARY_FLOAT => Some(FloatType) // Value for OracleTypes.BINARY_FLOAT case BINARY_DOUBLE => Some(DoubleType) // Value for OracleTypes.BINARY_DOUBLE + // scalastyle:off line.size.limit + // According to the documentation for Oracle Database 19c: + // "Values of the ROWID pseudocolumn are strings representing the address of each row." + // https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Data-Types.html#GUID-AEF1FE4C-2DE5-4BE7-BB53-83AD8F1E34EF + // scalastyle:on line.size.limit + case Types.ROWID => Some(StringType) case _ => None } } From d3dbe1a9076c8a76be0590ca071bfbec6114813b Mon Sep 17 00:00:00 2001 From: iRakson Date: Thu, 1 Oct 2020 20:50:16 +0900 Subject: [PATCH 12/59] [SQL][DOC][MINOR] Corrects input table names in the examples of CREATE FUNCTION doc ### What changes were proposed in this pull request? Fix Typo ### Why are the changes needed? To maintain consistency. Correct table name should be used for SELECT command. ### Does this PR introduce _any_ user-facing change? Yes. Now CREATE FUNCTION doc will show the correct name of table. ### How was this patch tested? Manually. Doc changes. Closes #29920 from iRakson/fixTypo. Authored-by: iRakson Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-ddl-create-function.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md index aa6c1fad7b56b..dfa4f4f8123d8 100644 --- a/docs/sql-ref-syntax-ddl-create-function.md +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -112,7 +112,7 @@ SHOW USER FUNCTIONS; +------------------+ -- Invoke the function. Every selected value should be incremented by 10. -SELECT simple_udf(c1) AS function_return_value FROM t1; +SELECT simple_udf(c1) AS function_return_value FROM test; +---------------------+ |function_return_value| +---------------------+ @@ -150,7 +150,7 @@ CREATE OR REPLACE FUNCTION simple_udf AS 'SimpleUdfR' USING JAR '/tmp/SimpleUdfR.jar'; -- Invoke the function. Every selected value should be incremented by 20. -SELECT simple_udf(c1) AS function_return_value FROM t1; +SELECT simple_udf(c1) AS function_return_value FROM test; +---------------------+ |function_return_value| +---------------------+ From 0963fcd848f62b4f2231dfcf67f9beabf927c21e Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 1 Oct 2020 08:37:07 -0500 Subject: [PATCH 13/59] [SPARK-33024][SQL] Fix CodeGen fallback issue of UDFSuite in Scala 2.13 ### What changes were proposed in this pull request? After `SPARK-32851` set `CODEGEN_FACTORY_MODE` to `CODEGEN_ONLY` of `sparkConf` in `SharedSparkSessionBase` to construction `SparkSession` in test, the test suite `SPARK-32459: UDF should not fail on WrappedArray` in s.sql.UDFSuite exposed a codegen fallback issue in Scala 2.13 as follow: ``` - SPARK-32459: UDF should not fail on WrappedArray *** FAILED *** Caused by: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, Column 99: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 47, Column 99: No applicable constructor/method found for zero actual parameters; candidates are: "public scala.collection.mutable.Builder scala.collection.mutable.ArraySeq$.newBuilder(java.lang.Object)", "public scala.collection.mutable.Builder scala.collection.mutable.ArraySeq$.newBuilder(scala.reflect.ClassTag)", "public abstract scala.collection.mutable.Builder scala.collection.EvidenceIterableFactory.newBuilder(java.lang.Object)" ``` The root cause is `WrappedArray` represent `mutable.ArraySeq` in Scala 2.13 and has a different constructor of `newBuilder` method. The main change of is pr is add Scala 2.13 only code part to deal with `case match WrappedArray` in Scala 2.13. ### Why are the changes needed? We need to support a Scala 2.13 build ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Scala 2.12: Pass the Jenkins or GitHub Action - Scala 2.13: All tests passed. Do the following: ``` dev/change-scala-version.sh 2.13 mvn clean install -DskipTests -pl sql/core -Pscala-2.13 -am mvn test -pl sql/core -Pscala-2.13 ``` **Before** ``` Tests: succeeded 8540, failed 1, canceled 1, ignored 52, pending 0 *** 1 TEST FAILED *** ``` **After** ``` Tests: succeeded 8541, failed 0, canceled 1, ignored 52, pending 0 All tests passed. ``` Closes #29903 from LuciferYang/fix-udfsuite. Authored-by: yangjie01 Signed-off-by: Sean Owen --- .../expressions/objects/objects.scala | 53 ++++++++++++++----- 1 file changed, 39 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 7933d05c8dba4..9701420e65870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -22,7 +22,7 @@ import java.lang.reflect.{Method, Modifier} import scala.collection.JavaConverters._ import scala.collection.mutable.{Builder, IndexedSeq, WrappedArray} import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Properties, Try} import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.serializer._ @@ -916,19 +916,44 @@ case class MapObjects private( val (initCollection, addElement, getResult): (String, String => String, String) = customCollectionCls match { case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) => - // Scala WrappedArray - val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" - val builder = ctx.freshName("collectionBuilder") - ( - s""" - ${classOf[Builder[_, _]].getName} $builder = $getBuilder; - $builder.sizeHint($dataLength); - """, - (genValue: String) => s"$builder.$$plus$$eq($genValue);", - s"(${cls.getName}) ${classOf[WrappedArray[_]].getName}$$." + - s"MODULE$$.make(((${classOf[IndexedSeq[_]].getName})$builder" + - s".result()).toArray(scala.reflect.ClassTag$$.MODULE$$.Object()));" - ) + def doCodeGenForScala212 = { + // WrappedArray in Scala 2.12 + val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" + val builder = ctx.freshName("collectionBuilder") + ( + s""" + ${classOf[Builder[_, _]].getName} $builder = $getBuilder; + $builder.sizeHint($dataLength); + """, + (genValue: String) => s"$builder.$$plus$$eq($genValue);", + s"(${cls.getName}) ${classOf[WrappedArray[_]].getName}$$." + + s"MODULE$$.make(((${classOf[IndexedSeq[_]].getName})$builder" + + s".result()).toArray(scala.reflect.ClassTag$$.MODULE$$.Object()));" + ) + } + + def doCodeGenForScala213 = { + // In Scala 2.13, WrappedArray is mutable.ArraySeq and newBuilder method need + // a ClassTag type construction parameter + val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder(" + + s"scala.reflect.ClassTag$$.MODULE$$.Object())" + val builder = ctx.freshName("collectionBuilder") + ( + s""" + ${classOf[Builder[_, _]].getName} $builder = $getBuilder; + $builder.sizeHint($dataLength); + """, + (genValue: String) => s"$builder.$$plus$$eq($genValue);", + s"(${cls.getName})$builder.result();" + ) + } + + val scalaVersion = Properties.versionNumberString + if (scalaVersion.startsWith("2.12")) { + doCodeGenForScala212 + } else { + doCodeGenForScala213 + } case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || classOf[scala.collection.Set[_]].isAssignableFrom(cls) => // Scala sequence or set From 9c618b33084c8ff6f68e5183e2574ba368fb7758 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 1 Oct 2020 12:41:40 -0700 Subject: [PATCH 14/59] [SPARK-33047][BUILD] Upgrade hive-storage-api to 2.7.2 ### What changes were proposed in this pull request? This PR aims to upgrade Apache Hive `hive-storage-api` library from 2.7.1 to 2.7.2. ### Why are the changes needed? [storage-api 2.7.2](https://github.com/apache/hive/commits/rel/storage-release-2.7.2/storage-api) has the following extension and can be used when users uses a provided orc dependency. [HIVE-22959](https://github.com/apache/hive/commit/dade9919d904f8a4bff12a9130c150301a4713ed#diff-ccfc9dd7584117f531322cda3a29f3c3) : Extend storage-api to expose FilterContext [HIVE-23215](https://github.com/apache/hive/commit/361925d2f3675bb9c6566b615a4b53faee335385#diff-ccfc9dd7584117f531322cda3a29f3c3) : Make FilterContext and MutableFilterContext interfaces ### Does this PR introduce _any_ user-facing change? Yes. This is a dependency change. ### How was this patch tested? Pass the existing tests. Closes #29923 from dongjoon-hyun/SPARK-33047. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 6d1934b46261b..7b31bdd98ef26 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -92,7 +92,7 @@ hive-shims-0.23/2.3.7//hive-shims-0.23-2.3.7.jar hive-shims-common/2.3.7//hive-shims-common-2.3.7.jar hive-shims-scheduler/2.3.7//hive-shims-scheduler-2.3.7.jar hive-shims/2.3.7//hive-shims-2.3.7.jar -hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar +hive-storage-api/2.7.2//hive-storage-api-2.7.2.jar hive-vector-code-gen/2.3.7//hive-vector-code-gen-2.3.7.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 2e29d831b9e66..960ea5f836ddf 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -91,7 +91,7 @@ hive-shims-0.23/2.3.7//hive-shims-0.23-2.3.7.jar hive-shims-common/2.3.7//hive-shims-common-2.3.7.jar hive-shims-scheduler/2.3.7//hive-shims-scheduler-2.3.7.jar hive-shims/2.3.7//hive-shims-2.3.7.jar -hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar +hive-storage-api/2.7.2//hive-storage-api-2.7.2.jar hive-vector-code-gen/2.3.7//hive-vector-code-gen-2.3.7.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar diff --git a/pom.xml b/pom.xml index 873daaa6161ee..421d932cef5fa 100644 --- a/pom.xml +++ b/pom.xml @@ -237,7 +237,7 @@ compile compile provided - 2.7.1 + 2.7.2 compile compile compile From e62d24717eb774f1c7adfd0fbe39640b96bc661d Mon Sep 17 00:00:00 2001 From: ulysses Date: Thu, 1 Oct 2020 15:58:01 -0400 Subject: [PATCH 15/59] [SPARK-32585][SQL] Support scala enumeration in ScalaReflection ### What changes were proposed in this pull request? Add code in `ScalaReflection` to support scala enumeration and make enumeration type as string type in Spark. ### Why are the changes needed? We support java enum but failed with scala enum, it's better to keep the same behavior. Here is a example. ``` package test object TestEnum extends Enumeration { type TestEnum = Value val E1, E2, E3 = Value } import TestEnum._ case class TestClass(i: Int, e: TestEnum) { } import test._ Seq(TestClass(1, TestEnum.E1)).toDS ``` Before this PR ``` Exception in thread "main" java.lang.UnsupportedOperationException: No Encoder found for test.TestEnum.TestEnum - field (class: "scala.Enumeration.Value", name: "e") - root class: "test.TestClass" at org.apache.spark.sql.catalyst.ScalaReflection$.$anonfun$serializerFor$1(ScalaReflection.scala:567) at scala.reflect.internal.tpe.TypeConstraints$UndoLog.undo(TypeConstraints.scala:69) at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects(ScalaReflection.scala:882) at org.apache.spark.sql.catalyst.ScalaReflection.cleanUpReflectionObjects$(ScalaReflection.scala:881) ``` After this PR `org.apache.spark.sql.Dataset[test.TestClass] = [i: int, e: string]` ### Does this PR introduce _any_ user-facing change? Yes, user can make case class which include scala enumeration field as dataset. ### How was this patch tested? Add test. Closes #29403 from ulysses-you/SPARK-32585. Authored-by: ulysses Signed-off-by: Tathagata Das --- .../spark/sql/catalyst/ScalaReflection.scala | 28 +++++++++++++++++++ .../sql/catalyst/ScalaReflectionSuite.scala | 15 ++++++++++ .../encoders/ExpressionEncoderSuite.scala | 10 ++++++- .../org/apache/spark/sql/DatasetSuite.scala | 15 +++++++++- 4 files changed, 66 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index a9c8b0bf4df2c..c65e181181e83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.util.Utils /** @@ -377,6 +378,23 @@ object ScalaReflection extends ScalaReflection { expressions.Literal.create(null, ObjectType(cls)), newInstance ) + + case t if isSubtype(t, localTypeOf[Enumeration#Value]) => + // package example + // object Foo extends Enumeration { + // type Foo = Value + // val E1, E2 = Value + // } + // the fullName of tpe is example.Foo.Foo, but we need example.Foo so that + // we can call example.Foo.withName to deserialize string to enumeration. + val parent = t.asInstanceOf[TypeRef].pre.typeSymbol.asClass + val cls = mirror.runtimeClass(parent) + StaticInvoke( + cls, + ObjectType(getClassFromType(t)), + "withName", + createDeserializerForString(path, false) :: Nil, + returnNullable = false) } } @@ -561,6 +579,14 @@ object ScalaReflection extends ScalaReflection { } createSerializerForObject(inputObject, fields) + case t if isSubtype(t, localTypeOf[Enumeration#Value]) => + createSerializerForString( + Invoke( + inputObject, + "toString", + ObjectType(classOf[java.lang.String]), + returnNullable = false)) + case _ => throw new UnsupportedOperationException( s"No Encoder found for $tpe\n" + walkedTypePath) @@ -738,6 +764,8 @@ object ScalaReflection extends ScalaReflection { val Schema(dataType, nullable) = schemaFor(fieldType) StructField(fieldName, dataType, nullable) }), nullable = true) + case t if isSubtype(t, localTypeOf[Enumeration#Value]) => + Schema(StringType, nullable = true) case other => throw new UnsupportedOperationException(s"Schema for type $other is not supported") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index b981a50499bf5..e8c7aed6d72ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.FooEnum.FooEnum import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, If, SpecificInternalRow, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, NewInstance} @@ -90,6 +91,13 @@ case class FooWithAnnotation(f1: String @FooAnnotation, f2: Option[String] @FooA case class SpecialCharAsFieldData(`field.1`: String, `field 2`: String) +object FooEnum extends Enumeration { + type FooEnum = Value + val E1, E2 = Value +} + +case class FooClassWithEnum(i: Int, e: FooEnum) + object TestingUDT { @SQLUserDefinedType(udt = classOf[NestedStructUDT]) class NestedStruct(val a: Integer, val b: Long, val c: Double) @@ -437,4 +445,11 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("f2", StringType)))) assert(deserializerFor[FooWithAnnotation].dataType == ObjectType(classOf[FooWithAnnotation])) } + + test("SPARK-32585: Support scala enumeration in ScalaReflection") { + assert(serializerFor[FooClassWithEnum].dataType == StructType(Seq( + StructField("i", IntegerType, false), + StructField("e", StringType, true)))) + assert(deserializerFor[FooClassWithEnum].dataType == ObjectType(classOf[FooClassWithEnum])) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 6a094d4aaddae..f2598a925e08e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.{Encoder, Encoders} -import org.apache.spark.sql.catalyst.{OptionalData, PrimitiveData} +import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, OptionalData, PrimitiveData} import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.AttributeReference @@ -389,6 +389,14 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes assert(e.getMessage.contains("tuple with more than 22 elements are not supported")) } + encodeDecodeTest((1, FooEnum.E1), "Tuple with Int and scala Enum") + encodeDecodeTest((null, FooEnum.E1, FooEnum.E2), "Tuple with Null and scala Enum") + encodeDecodeTest(Seq(FooEnum.E1, null), "Seq with scala Enum") + encodeDecodeTest(Map("key" -> FooEnum.E1), "Map with String key and scala Enum") + encodeDecodeTest(Map(FooEnum.E1 -> "value"), "Map with scala Enum key and String value") + encodeDecodeTest(FooClassWithEnum(1, FooEnum.E1), "case class with Int and scala Enum") + encodeDecodeTest(FooEnum.E1, "scala Enum") + // Scala / Java big decimals ---------------------------------------------------------- encodeDecodeTest(BigDecimal(("9" * 20) + "." + "9" * 18), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 4923e8b556907..3c914ae043677 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.exceptions.TestFailedException import org.scalatest.prop.TableDrivenPropertyChecks._ import org.apache.spark.{SparkException, TaskContext} -import org.apache.spark.sql.catalyst.ScroogeLikeExample +import org.apache.spark.sql.catalyst.{FooClassWithEnum, FooEnum, ScroogeLikeExample} import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.util.sideBySide @@ -1926,6 +1926,19 @@ class DatasetSuite extends QueryTest } } } + + test("SPARK-32585: Support scala enumeration in ScalaReflection") { + checkDataset( + Seq(FooClassWithEnum(1, FooEnum.E1), FooClassWithEnum(2, FooEnum.E2)).toDS(), + Seq(FooClassWithEnum(1, FooEnum.E1), FooClassWithEnum(2, FooEnum.E2)): _* + ) + + // test null + checkDataset( + Seq(FooClassWithEnum(1, null), FooClassWithEnum(2, FooEnum.E2)).toDS(), + Seq(FooClassWithEnum(1, null), FooClassWithEnum(2, FooEnum.E2)): _* + ) + } } object AssertExecutionId { From 005999721f103bce653c39a0001cba7f2e04b7c8 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 1 Oct 2020 18:01:23 -0500 Subject: [PATCH 16/59] [SPARK-33046][DOCS] Update how to build doc for Scala 2.13 with sbt ### What changes were proposed in this pull request? This PR fixes the description how to build Spark for Scala 2.13 with sbt. In the current doc, how to build Spark for Scala 2.13 with sbt is described like: ![scala-2 13-build-before](https://user-images.githubusercontent.com/4736016/94816248-80c3e900-0436-11eb-9bc2-99af5786971a.png) But build fails with this command because scala-2.13 profile is not enabled and scala-parallel-collections is absent. ``` [error] /home/kou/work/oss/spark-scala-2.13/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala:23: object parallel is not a member of package collection ``` The correct command should be: ``` build/sbt -Pspark-2.13 compile ``` ### Why are the changes needed? The build command is wrong. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I checked that `sbt -Pspark-2.13` is correct with the following command: ``` build/sbt -Dscala.version=2.13.3 -Phive -Phive-thriftserver -Pyarn -Pkubernetes compile ``` I also build the modified doc and checked the generated html: ![spark-scala-2 13-build-doc-after](https://user-images.githubusercontent.com/4736016/94869259-f2745500-047f-11eb-89e5-20816f3ed24d.png) Closes #29921 from sarutak/fix-scala-2.13-build-doc. Authored-by: Kousuke Saruta Signed-off-by: Sean Owen --- docs/building-spark.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 3d12a60e2b974..73c527b7a5ed6 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -265,15 +265,13 @@ Change the major Scala version using (e.g. 2.13): ./dev/change-scala-version.sh 2.13 -For Maven, please enable the profile (e.g. 2.13): +Enable the profile (e.g. 2.13): + # For Maven ./build/mvn -Pscala-2.13 compile -For SBT, specify a complete scala version using (e.g. 2.13.0): - - ./build/sbt -Dscala.version=2.13.0 - -Otherwise, the sbt-pom-reader plugin will use the `scala.version` specified in the spark-parent pom. + # For sbt + ./build/sbt -Pscala-2.13 compile ## Running Jenkins tests with Github Enterprise From 8657742ec7570c8292ed45629fc61b9791f28796 Mon Sep 17 00:00:00 2001 From: Shruti Gumma Date: Thu, 1 Oct 2020 16:33:19 -0700 Subject: [PATCH 17/59] [SPARK-32996][WEB-UI][FOLLOWUP] Move ExecutorSummarySuite to proper path ### What changes were proposed in this pull request? This change updates the test file location in #29872 to proper path. ### Why are the changes needed? ExecutorSummarySuite.scala should be in core/src/test/scala instead of core/src/test/java. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit tests Closes #29926 from shrutig/SPARK-32996. Authored-by: Shruti Gumma Signed-off-by: Liang-Chi Hsieh --- .../org/apache/spark/status/api/v1/ExecutorSummarySuite.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/src/test/{java => scala}/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala (100%) diff --git a/core/src/test/java/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala similarity index 100% rename from core/src/test/java/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala rename to core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala From d6f3138352042e33a2291e11c325b8eadb8dd5f2 Mon Sep 17 00:00:00 2001 From: Cheng Su Date: Fri, 2 Oct 2020 09:01:15 +0900 Subject: [PATCH 18/59] [SPARK-32859][SQL] Introduce physical rule to decide bucketing dynamically ### What changes were proposed in this pull request? This PR is to add support to decide bucketed table scan dynamically based on actual query plan. Currently bucketing is enabled by default (`spark.sql.sources.bucketing.enabled`=true), so for all bucketed tables in the query plan, we will use bucket table scan (all input files per the bucket will be read by same task). This has the drawback that if the bucket table scan is not benefitting at all (no join/groupby/etc in the query), we don't need to use bucket table scan as it would restrict the # of tasks to be # of buckets and might hurt parallelism. The feature is to add a physical plan rule right after `EnsureRequirements`: The rule goes through plan nodes. For all operators which has "interesting partition" (i.e., require `ClusteredDistribution` or `HashClusteredDistribution`), check if the sub-plan for operator has `Exchange` and bucketed table scan (and only allow certain operators in plan (i.e. `Scan/Filter/Project/Sort/PartialAgg/etc`.), see details in `DisableUnnecessaryBucketedScan.disableBucketWithInterestingPartition`). If yes, disable the bucketed table scan in the sub-plan. In addition, disabling bucketed table scan if there's operator with interesting partition along the sub-plan. Why the algorithm works is that if there's a shuffle between the bucketed table scan and operator with interesting partition, then bucketed table scan partitioning will be destroyed by the shuffle operator in the middle, and we don't need bucketed table scan for sure. The idea of "interesting partition" is inspired from "interesting order" in "Access Path Selection in a Relational Database Management System"(http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf), after discussion with cloud-fan . ### Why are the changes needed? To avoid unnecessary bucketed scan in the query, and this is prerequisite for https://github.com/apache/spark/pull/29625 (decide bucketed sorted scan dynamically will be added later in that PR). ### Does this PR introduce _any_ user-facing change? A new config `spark.sql.sources.bucketing.autoBucketedScan.enabled` is introduced which set to false by default (the rule is disabled by default as it can regress cached bucketed table query, see discussion in https://github.com/apache/spark/pull/29804#issuecomment-701151447). User can opt-in/opt-out by enabling/disabling the config, as we found in prod, some users rely on assumption of # of tasks == # of buckets when reading bucket table to precisely control # of tasks. This is a bad assumption but it does happen on our side, so leave a config here to allow them opt-out for the feature. ### How was this patch tested? Added unit tests in `DisableUnnecessaryBucketedScanSuite.scala` Closes #29804 from c21/bucket-rule. Authored-by: Cheng Su Signed-off-by: Takeshi Yamamuro --- .../optimizer/CostBasedJoinReorder.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 13 ++ .../sql/execution/DataSourceScanExec.scala | 37 +-- .../spark/sql/execution/QueryExecution.scala | 3 +- .../DisableUnnecessaryBucketedScan.scala | 161 +++++++++++++ .../apache/spark/sql/DataFrameJoinSuite.scala | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 2 +- .../DisableUnnecessaryBucketedScanSuite.scala | 221 ++++++++++++++++++ ...saryBucketedScanWithHiveSupportSuite.scala | 31 +++ 9 files changed, 454 insertions(+), 18 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 8b019f35263f3..45541051a6b13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -114,7 +114,7 @@ case class OrderedJoin( /** * Reorder the joins using a dynamic programming algorithm. This implementation is based on the * paper: Access Path Selection in a Relational Database Management System. - * http://www.inf.ed.ac.uk/teaching/courses/adbs/AccessPath.pdf + * https://dl.acm.org/doi/10.1145/582095.582099 * * First we put all items (basic joined nodes) into level 0, then we build all two-way joins * at level 1 from plans at level 0 (single items), then build all 3-way joins from plans diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0d1a3e365c918..18ffc655b2174 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -951,6 +951,17 @@ object SQLConf { .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) + val AUTO_BUCKETED_SCAN_ENABLED = + buildConf("spark.sql.sources.bucketing.autoBucketedScan.enabled") + .doc("When true, decide whether to do bucketed scan on input tables based on query plan " + + "automatically. Do not use bucketed scan if 1. query does not have operators to utilize " + + "bucketing (e.g. join, group-by, etc), or 2. there's an exchange operator between these " + + s"operators and table scan. Note when '${BUCKETING_ENABLED.key}' is set to " + + "false, this configuration does not take any effect.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") .internal() .doc("When false, we will throw an error if a query contains a cartesian product without " + @@ -3164,6 +3175,8 @@ class SQLConf extends Serializable with Logging { def bucketingMaxBuckets: Int = getConf(SQLConf.BUCKETING_MAX_BUCKETS) + def autoBucketedScanEnabled: Boolean = getConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED) + def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 1b9ca63ea21d3..45d28ddb42fc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -156,7 +156,9 @@ case class RowDataSourceScanExec( * @param optionalBucketSet Bucket ids for bucket pruning. * @param optionalNumCoalescedBuckets Number of coalesced buckets. * @param dataFilters Filters on non-partition columns. - * @param tableIdentifier identifier for the table in the metastore. + * @param tableIdentifier Identifier for the table in the metastore. + * @param disableBucketedScan Disable bucketed scan based on physical query plan, see rule + * [[DisableUnnecessaryBucketedScan]] for details. */ case class FileSourceScanExec( @transient relation: HadoopFsRelation, @@ -166,7 +168,8 @@ case class FileSourceScanExec( optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], dataFilters: Seq[Expression], - tableIdentifier: Option[TableIdentifier]) + tableIdentifier: Option[TableIdentifier], + disableBucketedScan: Boolean = false) extends DataSourceScanExec { // Note that some vals referring the file-based relation are lazy intentionally @@ -257,7 +260,8 @@ case class FileSourceScanExec( // exposed for testing lazy val bucketedScan: Boolean = { - if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined) { + if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined + && !disableBucketedScan) { val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) bucketColumns.size == spec.bucketColumnNames.size @@ -348,20 +352,23 @@ case class FileSourceScanExec( "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) - val withSelectedBucketsCount = relation.bucketSpec.map { spec => - val numSelectedBuckets = optionalBucketSet.map { b => - b.cardinality() + // TODO(SPARK-32986): Add bucketed scan info in explain output of FileSourceScanExec + if (bucketedScan) { + relation.bucketSpec.map { spec => + val numSelectedBuckets = optionalBucketSet.map { b => + b.cardinality() + } getOrElse { + spec.numBuckets + } + metadata + ("SelectedBucketsCount" -> + (s"$numSelectedBuckets out of ${spec.numBuckets}" + + optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) } getOrElse { - spec.numBuckets + metadata } - metadata + ("SelectedBucketsCount" -> - (s"$numSelectedBuckets out of ${spec.numBuckets}" + - optionalNumCoalescedBuckets.map { b => s" (Coalesced to $b)"}.getOrElse(""))) - } getOrElse { + } else { metadata } - - withSelectedBucketsCount } override def verboseStringWithOperatorId(): String = { @@ -539,6 +546,7 @@ case class FileSourceScanExec( .getOrElse(sys.error(s"Invalid bucket file ${f.filePath}")) } + // TODO(SPARK-32985): Decouple bucket filter pruning and bucketed table scan val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { val bucketSet = optionalBucketSet.get filesGroupedToBuckets.filter { @@ -624,6 +632,7 @@ case class FileSourceScanExec( optionalBucketSet, optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), - None) + None, + disableBucketedScan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index dca2c5b16e8d5..a056500fa361a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.{AdaptiveExecutionContext, InsertAdaptiveSparkPlan} -import org.apache.spark.sql.execution.bucketing.CoalesceBucketsInJoin +import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableUnnecessaryBucketedScan} import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -344,6 +344,7 @@ object QueryExecution { PlanSubqueries(sparkSession), RemoveRedundantProjects(sparkSession.sessionState.conf), EnsureRequirements(sparkSession.sessionState.conf), + DisableUnnecessaryBucketedScan(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, sparkSession.sessionState.columnarRules), CollapseCodegenStages(sparkSession.sessionState.conf), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala new file mode 100644 index 0000000000000..9b4f898df00b6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/bucketing/DisableUnnecessaryBucketedScan.scala @@ -0,0 +1,161 @@ +/* + * 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.spark.sql.execution.bucketing + +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashClusteredDistribution} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SortExec, SparkPlan} +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.internal.SQLConf + +/** + * Disable unnecessary bucketed table scan based on actual physical query plan. + * NOTE: this rule is designed to be applied right after [[EnsureRequirements]], + * where all [[ShuffleExchangeExec]] and [[SortExec]] have been added to plan properly. + * + * When BUCKETING_ENABLED and AUTO_BUCKETED_SCAN_ENABLED are set to true, go through + * query plan to check where bucketed table scan is unnecessary, and disable bucketed table + * scan if: + * + * 1. The sub-plan from root to bucketed table scan, does not contain + * [[hasInterestingPartition]] operator. + * + * 2. The sub-plan from the nearest downstream [[hasInterestingPartition]] operator + * to the bucketed table scan, contains only [[isAllowedUnaryExecNode]] operators + * and at least one [[Exchange]]. + * + * Examples: + * 1. no [[hasInterestingPartition]] operator: + * Project + * | + * Filter + * | + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * 2. join: + * SortMergeJoin(t1.i = t2.j) + * / \ + * Sort(i) Sort(j) + * / \ + * Shuffle(i) Scan(t2: i, j) + * / (bucketed on column j, enable bucketed scan) + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * 3. aggregate: + * HashAggregate(i, ..., Final) + * | + * Shuffle(i) + * | + * HashAggregate(i, ..., Partial) + * | + * Filter + * | + * Scan(t1: i, j) + * (bucketed on column j, DISABLE bucketed scan) + * + * The idea of [[hasInterestingPartition]] is inspired from "interesting order" in + * the paper "Access Path Selection in a Relational Database Management System" + * (https://dl.acm.org/doi/10.1145/582095.582099). + */ +case class DisableUnnecessaryBucketedScan(conf: SQLConf) extends Rule[SparkPlan] { + + /** + * Disable bucketed table scan with pre-order traversal of plan. + * + * @param withInterestingPartition The traversed plan has operator with interesting partition. + * @param withExchange The traversed plan has [[Exchange]] operator. + * @param withAllowedNode The traversed plan has only [[isAllowedUnaryExecNode]] operators. + */ + private def disableBucketWithInterestingPartition( + plan: SparkPlan, + withInterestingPartition: Boolean, + withExchange: Boolean, + withAllowedNode: Boolean): SparkPlan = { + plan match { + case p if hasInterestingPartition(p) => + // Operator with interesting partition, propagates `withInterestingPartition` as true + // to its children, and resets `withExchange` and `withAllowedNode`. + p.mapChildren(disableBucketWithInterestingPartition(_, true, false, true)) + case exchange: Exchange => + // Exchange operator propagates `withExchange` as true to its child. + exchange.mapChildren(disableBucketWithInterestingPartition( + _, withInterestingPartition, true, withAllowedNode)) + case scan: FileSourceScanExec => + if (isBucketedScanWithoutFilter(scan)) { + if (!withInterestingPartition || (withExchange && withAllowedNode)) { + scan.copy(disableBucketedScan = true) + } else { + scan + } + } else { + scan + } + case o => + o.mapChildren(disableBucketWithInterestingPartition( + _, + withInterestingPartition, + withExchange, + withAllowedNode && isAllowedUnaryExecNode(o))) + } + } + + private def hasInterestingPartition(plan: SparkPlan): Boolean = { + plan.requiredChildDistribution.exists { + case _: ClusteredDistribution | _: HashClusteredDistribution => true + case _ => false + } + } + + /** + * Check if the operator is allowed single-child operator. + * We may revisit this method later as we probably can + * remove this restriction to allow arbitrary operator between + * bucketed table scan and operator with interesting partition. + */ + private def isAllowedUnaryExecNode(plan: SparkPlan): Boolean = { + plan match { + case _: SortExec | _: ProjectExec | _: FilterExec => true + case partialAgg: BaseAggregateExec => + partialAgg.requiredChildDistributionExpressions.isEmpty + case _ => false + } + } + + private def isBucketedScanWithoutFilter(scan: FileSourceScanExec): Boolean = { + // Do not disable bucketed table scan if it has filter pruning, + // because bucketed table scan is still useful here to save CPU/IO cost with + // only reading selected bucket files. + scan.bucketedScan && scan.optionalBucketSet.isEmpty + } + + def apply(plan: SparkPlan): SparkPlan = { + lazy val hasBucketedScanWithoutFilter = plan.find { + case scan: FileSourceScanExec => isBucketedScanWithoutFilter(scan) + case _ => false + }.isDefined + + if (!conf.bucketingEnabled || !conf.autoBucketedScanEnabled || !hasBucketedScanWithoutFilter) { + plan + } else { + disableBucketWithInterestingPartition(plan, false, false, true) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index b463a76a74026..14d03a30453ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -348,7 +348,7 @@ class DataFrameJoinSuite extends QueryTest } assert(broadcastExchanges.size == 1) val tables = broadcastExchanges.head.collect { - case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent)) => tableIdent + case FileSourceScanExec(_, _, _, _, _, _, _, Some(tableIdent), _) => tableIdent } assert(tables.size == 1) assert(tables.head === TableIdentifier(table1Name, Some(dbName))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a21c461e84588..73b23496de515 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1314,7 +1314,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark // need to execute the query before we can examine fs.inputRDDs() assert(stripAQEPlan(df.queryExecution.executedPlan) match { case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( - fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _)))) => + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _, _, _)))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala new file mode 100644 index 0000000000000..1c258bc0dadb9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -0,0 +1,221 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} + +class DisableUnnecessaryBucketedScanWithoutHiveSupportSuite + extends DisableUnnecessaryBucketedScanSuite + with SharedSparkSession { + + protected override def beforeAll(): Unit = { + super.beforeAll() + assert(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + } +} + +abstract class DisableUnnecessaryBucketedScanSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + private lazy val df1 = + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k").as("df1") + private lazy val df2 = + (0 until 50).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k").as("df2") + + private def checkDisableBucketedScan( + query: String, + expectedNumScanWithAutoScanEnabled: Int, + expectedNumScanWithAutoScanDisabled: Int): Unit = { + + def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { + val plan = sql(query).queryExecution.executedPlan + val bucketedScan = plan.collect { case s: FileSourceScanExec if s.bucketedScan => s } + assert(bucketedScan.length == expectedNumBucketedScan) + } + + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "true") { + checkNumBucketedScan(query, expectedNumScanWithAutoScanEnabled) + val result = sql(query).collect() + + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "false") { + checkNumBucketedScan(query, expectedNumScanWithAutoScanDisabled) + checkAnswer(sql(query), result) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - basic test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t3") + + Seq( + // Read bucketed table + ("SELECT * FROM t1", 0, 1), + ("SELECT i FROM t1", 0, 1), + ("SELECT j FROM t1", 0, 0), + // Filter on bucketed column + ("SELECT * FROM t1 WHERE i = 1", 1, 1), + // Filter on non-bucketed column + ("SELECT * FROM t1 WHERE j = 1", 0, 1), + // Join with same buckets + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 2, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 2, 2), + // Join with different buckets + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i", 1, 2), + // Join on non-bucketed column + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.j", 1, 2), + ("SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + ("SELECT /*+ shuffle_hash(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.j = t2.j", 0, 2), + // Aggregate on bucketed column + ("SELECT SUM(i) FROM t1 GROUP BY i", 1, 1), + // Aggregate on non-bucketed column + ("SELECT SUM(i) FROM t1 GROUP BY j", 0, 1), + ("SELECT j, SUM(i), COUNT(j) FROM t1 GROUP BY j", 0, 1) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i").saveAsTable("t3") + + Seq( + // Multiple joins on bucketed columns + (""" + SELECT /*+ broadcast(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 0, 3), + (""" + SELECT /*+ broadcast(t1) merge(t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + (""" + SELECT /*+ merge(t1) broadcast(t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.i AND t2.i = t3.i + """.stripMargin, 2, 3), + // Multiple joins on non-bucketed columns + (""" + SELECT /*+ broadcast(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.j AND t2.j = t3.i + """.stripMargin, 0, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.i = t2.j AND t2.j = t3.i + """.stripMargin, 1, 3), + (""" + SELECT /*+ merge(t1, t3)*/ * FROM t1 JOIN t2 JOIN t3 + ON t1.j = t2.j AND t2.j = t3.j + """.stripMargin, 0, 3) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t2") + df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t3") + + Seq( + // Filter on bucketed columns + ("SELECT * FROM t1 WHERE i = 1", 0, 1), + ("SELECT * FROM t1 WHERE i = 1 AND j = 1", 0, 1), + // Join on bucketed columns + (""" + SELECT /*+ broadcast(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j + """.stripMargin, 0, 2), + (""" + SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i AND t1.j = t2.j + """.stripMargin, 2, 2), + (""" + SELECT /*+ merge(t1)*/ * FROM t1 JOIN t3 ON t1.i = t3.i AND t1.j = t3.j + """.stripMargin, 1, 2), + ("SELECT /*+ merge(t1)*/ * FROM t1 JOIN t2 ON t1.i = t2.i", 0, 2), + // Aggregate on bucketed columns + ("SELECT i, j, COUNT(*) FROM t1 GROUP BY i, j", 1, 1), + ("SELECT i, COUNT(i) FROM t1 GROUP BY i", 0, 0), + ("SELECT i, COUNT(j) FROM t1 GROUP BY i", 0, 1) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } + + test("SPARK-32859: disable unnecessary bucketed table scan - other operators test") { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") + df1.write.format("parquet").saveAsTable("t3") + + Seq( + // Operator with interesting partition not in sub-plan + (""" + SELECT t1.i FROM t1 + UNION ALL + (SELECT t2.i FROM t2 GROUP BY t2.i) + """.stripMargin, 1, 2), + // Non-allowed operator in sub-plan + (""" + SELECT COUNT(*) + FROM (SELECT t1.i FROM t1 UNION ALL SELECT t2.i FROM t2) + GROUP BY i + """.stripMargin, 2, 2), + // Multiple [[Exchange]] in sub-plan + (""" + SELECT j, SUM(i), COUNT(*) FROM t1 GROUP BY j + DISTRIBUTE BY j + """.stripMargin, 0, 1), + (""" + SELECT j, COUNT(*) + FROM (SELECT i, j FROM t1 DISTRIBUTE BY i, j) + GROUP BY j + """.stripMargin, 0, 1), + // No bucketed table scan in plan + (""" + SELECT j, COUNT(*) + FROM (SELECT t1.j FROM t1 JOIN t3 ON t1.j = t3.j) + GROUP BY j + """.stripMargin, 0, 0) + ).foreach { case (query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) => + checkDisableBucketedScan(query, numScanWithAutoScanEnabled, numScanWithAutoScanDisabled) + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala new file mode 100644 index 0000000000000..30eb93cb5c3e8 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanWithHiveSupportSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION + +class DisableUnnecessaryBucketedScanWithHiveSupportSuite + extends DisableUnnecessaryBucketedScanSuite + with TestHiveSingleton { + + protected override def beforeAll(): Unit = { + super.beforeAll() + assert(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + } +} From 991f7e81d46820f6e097fcf92c025689b677491f Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 2 Oct 2020 13:04:40 +0900 Subject: [PATCH 19/59] [SPARK-32001][SQL] Create JDBC authentication provider developer API ### What changes were proposed in this pull request? At the moment only the baked in JDBC connection providers can be used but there is a need to support additional databases and use-cases. In this PR I'm proposing a new developer API name `JdbcConnectionProvider`. To show how an external JDBC connection provider can be implemented I've created an example [here](https://github.com/gaborgsomogyi/spark-jdbc-connection-provider). The PR contains the following changes: * Added connection provider developer API * Made JDBC connection providers constructor to noarg => needed to load them w/ service loader * Connection providers are now loaded w/ service loader * Added tests to load providers independently * Moved `SecurityConfigurationLock` into a central place because other areas will change global JVM security config ### Why are the changes needed? No custom authentication possibility. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? * Existing + additional unit tests * Docker integration tests * Tested manually the newly created external JDBC connection provider Closes #29024 from gaborgsomogyi/SPARK-32001. Authored-by: Gabor Somogyi Signed-off-by: HyukjinKwon --- .../security/SecurityConfigurationLock.scala | 24 ++++++ .../sql/jdbc/DB2KrbIntegrationSuite.scala | 2 +- ...ache.spark.sql.jdbc.JdbcConnectionProvider | 6 ++ .../datasources/jdbc/JDBCOptions.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../connection/BasicConnectionProvider.scala | 30 +++++-- .../jdbc/connection/ConnectionProvider.scala | 83 ++++++++----------- .../connection/DB2ConnectionProvider.scala | 39 +++++---- .../connection/MSSQLConnectionProvider.scala | 41 +++++---- .../MariaDBConnectionProvider.scala | 18 ++-- .../connection/OracleConnectionProvider.scala | 39 +++++---- .../PostgresConnectionProvider.scala | 17 ++-- .../connection/SecureConnectionProvider.scala | 42 ++++++---- .../sql/jdbc/JdbcConnectionProvider.scala | 58 +++++++++++++ ...ache.spark.sql.jdbc.JdbcConnectionProvider | 1 + .../connection/ConnectionProviderSuite.scala | 39 ++++++--- .../ConnectionProviderSuiteBase.scala | 17 ++-- .../DB2ConnectionProviderSuite.scala | 6 +- ...ntentionallyFaultyConnectionProvider.scala | 33 ++++++++ .../MSSQLConnectionProviderSuite.scala | 42 +++++----- .../MariaDBConnectionProviderSuite.scala | 6 +- .../OracleConnectionProviderSuite.scala | 7 +- .../PostgresConnectionProviderSuite.scala | 16 ++-- 23 files changed, 355 insertions(+), 217 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala create mode 100644 sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala diff --git a/core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala b/core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala new file mode 100644 index 0000000000000..0741a8c1580df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/SecurityConfigurationLock.scala @@ -0,0 +1,24 @@ +/* + * 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.spark.security + +/** + * There are cases when global JVM security configuration must be modified. + * In order to avoid race the modification must be synchronized with this. + */ +object SecurityConfigurationLock diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index fc88985cf2ec7..fa5ce2d106a10 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -54,7 +54,7 @@ class DB2KrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { JDBCOptions.JDBC_KEYTAB -> keytabFileName, JDBCOptions.JDBC_PRINCIPAL -> principal )) - new DB2ConnectionProvider(null, options).getAdditionalProperties() + new DB2ConnectionProvider().getAdditionalProperties(options) } override def beforeContainerStart( diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider new file mode 100644 index 0000000000000..6e42517a6d40c --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider @@ -0,0 +1,6 @@ +org.apache.spark.sql.execution.datasources.jdbc.connection.BasicConnectionProvider +org.apache.spark.sql.execution.datasources.jdbc.connection.DB2ConnectionProvider +org.apache.spark.sql.execution.datasources.jdbc.connection.MariaDBConnectionProvider +org.apache.spark.sql.execution.datasources.jdbc.connection.MSSQLConnectionProvider +org.apache.spark.sql.execution.datasources.jdbc.connection.PostgresConnectionProvider +org.apache.spark.sql.execution.datasources.jdbc.connection.OracleConnectionProvider \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 9e0438c0016bd..e6fff8dbdbd7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap * Options for the JDBC data source. */ class JDBCOptions( - @transient val parameters: CaseInsensitiveMap[String]) + val parameters: CaseInsensitiveMap[String]) extends Serializable with Logging { import JDBCOptions._ @@ -209,7 +209,7 @@ class JDBCOptions( } class JdbcOptionsInWrite( - @transient override val parameters: CaseInsensitiveMap[String]) + override val parameters: CaseInsensitiveMap[String]) extends JDBCOptions(parameters) { import JDBCOptions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 5831c35c7e301..202f2e03b68d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -63,7 +63,7 @@ object JdbcUtils extends Logging { throw new IllegalStateException( s"Did not find registered driver with class $driverClass") } - val connection = ConnectionProvider.create(driver, options).getConnection() + val connection = ConnectionProvider.create(driver, options.parameters) require(connection != null, s"The driver could not open a JDBC connection. Check the URL: ${options.url}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala index 16b244cc617ce..a5f04649e6628 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/BasicConnectionProvider.scala @@ -18,18 +18,30 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import java.sql.{Connection, Driver} +import java.util.Properties -import scala.collection.JavaConverters._ - +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.jdbc.JdbcConnectionProvider + +private[jdbc] class BasicConnectionProvider extends JdbcConnectionProvider with Logging { + /** + * Additional properties for data connection (Data source property takes precedence). + */ + def getAdditionalProperties(options: JDBCOptions): Properties = new Properties() + + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = { + val jdbcOptions = new JDBCOptions(options) + jdbcOptions.keytab == null || jdbcOptions.principal == null + } -private[jdbc] class BasicConnectionProvider(driver: Driver, options: JDBCOptions) - extends ConnectionProvider { - def getConnection(): Connection = { - val properties = getAdditionalProperties() - options.asConnectionProperties.entrySet().asScala.foreach { e => - properties.put(e.getKey(), e.getValue()) + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + val properties = getAdditionalProperties(jdbcOptions) + options.foreach { case(k, v) => + properties.put(k, v) } - driver.connect(options.url, properties) + logDebug(s"JDBC connection initiated with URL: ${jdbcOptions.url} and properties: $properties") + driver.connect(jdbcOptions.url, properties) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index ce45be442ccc3..546756677edce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -18,60 +18,45 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import java.sql.{Connection, Driver} -import java.util.Properties +import java.util.ServiceLoader -import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions - -/** - * Connection provider which opens connection toward various databases (database specific instance - * needed). If kerberos authentication required then it's the provider's responsibility to set all - * the parameters. - */ -private[jdbc] trait ConnectionProvider { - /** - * Additional properties for data connection (Data source property takes precedence). - */ - def getAdditionalProperties(): Properties = new Properties() +import scala.collection.mutable - /** - * Opens connection toward the database. - */ - def getConnection(): Connection -} +import org.apache.spark.internal.Logging +import org.apache.spark.security.SecurityConfigurationLock +import org.apache.spark.sql.jdbc.JdbcConnectionProvider +import org.apache.spark.util.Utils private[jdbc] object ConnectionProvider extends Logging { - def create(driver: Driver, options: JDBCOptions): ConnectionProvider = { - if (options.keytab == null || options.principal == null) { - logDebug("No authentication configuration found, using basic connection provider") - new BasicConnectionProvider(driver, options) - } else { - logDebug("Authentication configuration found, using database specific connection provider") - options.driverClass match { - case PostgresConnectionProvider.driverClass => - logDebug("Postgres connection provider found") - new PostgresConnectionProvider(driver, options) - - case MariaDBConnectionProvider.driverClass => - logDebug("MariaDB connection provider found") - new MariaDBConnectionProvider(driver, options) - - case DB2ConnectionProvider.driverClass => - logDebug("DB2 connection provider found") - new DB2ConnectionProvider(driver, options) - - case MSSQLConnectionProvider.driverClass => - logDebug("MS SQL connection provider found") - new MSSQLConnectionProvider(driver, options) - - case OracleConnectionProvider.driverClass => - logDebug("Oracle connection provider found") - new OracleConnectionProvider(driver, options) - - case _ => - throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + - "Kerberos authentication") + private val providers = loadProviders() + + def loadProviders(): Seq[JdbcConnectionProvider] = { + val loader = ServiceLoader.load(classOf[JdbcConnectionProvider], + Utils.getContextOrSparkClassLoader) + val providers = mutable.ArrayBuffer[JdbcConnectionProvider]() + + val iterator = loader.iterator + while (iterator.hasNext) { + try { + val provider = iterator.next + logDebug(s"Loaded built in provider: $provider") + providers += provider + } catch { + case t: Throwable => + logError(s"Failed to load built in provider.", t) } } + // Seems duplicate but it's needed for Scala 2.13 + providers.toSeq + } + + def create(driver: Driver, options: Map[String, String]): Connection = { + val filteredProviders = providers.filter(_.canHandle(driver, options)) + require(filteredProviders.size == 1, + "JDBC connection initiated but not exactly one connection provider found which can handle " + + s"it. Found active providers: ${filteredProviders.mkString(", ")}") + SecurityConfigurationLock.synchronized { + filteredProviders.head.getConnection(driver, options) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala index 095821cf83890..ca82cdc561bef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala @@ -25,22 +25,25 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class DB2ConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = "JaasClient" - - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - DB2ConnectionProvider.super.getConnection() +private[sql] class DB2ConnectionProvider extends SecureConnectionProvider { + override val driverClass = "com.ibm.db2.jcc.DB2Driver" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "JaasClient" + + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + DB2ConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // 11 is the integer value for kerberos result.put("securityMechanism", new String("11")) @@ -48,14 +51,10 @@ private[sql] class DB2ConnectionProvider(driver: Driver, options: JDBCOptions) result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object DB2ConnectionProvider { - val driverClass = "com.ibm.db2.jcc.DB2Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala index 2950aa9b4db94..4e405b2187e56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala @@ -25,12 +25,11 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class MSSQLConnectionProvider( - driver: Driver, - options: JDBCOptions, - parserMethod: String = "parseAndMergeProperties" - ) extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider { + override val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" + val parserMethod: String = "parseAndMergeProperties" + + override def appEntry(driver: Driver, options: JDBCOptions): String = { val configName = "jaasConfigurationName" val appEntryDefault = "SQLJDBCDriver" @@ -58,18 +57,20 @@ private[sql] class MSSQLConnectionProvider( } } - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - MSSQLConnectionProvider.super.getConnection() + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + MSSQLConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // These props needed to reach internal kerberos authentication in the JDBC driver result.put("integratedSecurity", "true") @@ -77,8 +78,8 @@ private[sql] class MSSQLConnectionProvider( result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) /** * Couple of things to mention here (v8.2.2 client): * 1. MS SQL supports JAAS application name configuration @@ -87,11 +88,7 @@ private[sql] class MSSQLConnectionProvider( val entryUsesKeytab = configEntry != null && configEntry.exists(_.getOptions().get("useKeyTab") == "true") if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object MSSQLConnectionProvider { - val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 3c0286654a8ec..d5fe13bf0ca19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -21,14 +21,14 @@ import java.sql.Driver import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[jdbc] class MariaDBConnectionProvider extends SecureConnectionProvider { + override val driverClass = "org.mariadb.jdbc.Driver" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "Krb5ConnectorContext" - } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) /** * Couple of things to mention here (v2.5.4 client): * 1. MariaDB doesn't support JAAS application name configuration @@ -37,11 +37,7 @@ private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptio val entryUsesKeytab = configEntry != null && configEntry.exists(_.getOptions().get("useKeyTab") == "true") if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object MariaDBConnectionProvider { - val driverClass = "org.mariadb.jdbc.Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala index c2b71b35b8128..3defda3871765 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala @@ -25,22 +25,25 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[sql] class OracleConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = "kprb5module" - - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( - new PrivilegedExceptionAction[Connection]() { - override def run(): Connection = { - OracleConnectionProvider.super.getConnection() +private[sql] class OracleConnectionProvider extends SecureConnectionProvider { + override val driverClass = "oracle.jdbc.OracleDriver" + + override def appEntry(driver: Driver, options: JDBCOptions): String = "kprb5module" + + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) + .doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + OracleConnectionProvider.super.getConnection(driver, options) + } } - } - ) + ) } - override def getAdditionalProperties(): Properties = { + override def getAdditionalProperties(options: JDBCOptions): Properties = { val result = new Properties() // This prop is needed to turn on kerberos authentication in the JDBC driver. // The possible values can be found in AnoServices public interface @@ -49,14 +52,10 @@ private[sql] class OracleConnectionProvider(driver: Driver, options: JDBCOptions result } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object OracleConnectionProvider { - val driverClass = "oracle.jdbc.OracleDriver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala index fa9232e00bd88..dae8aea81f20a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala @@ -22,22 +22,19 @@ import java.util.Properties import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -private[jdbc] class PostgresConnectionProvider(driver: Driver, options: JDBCOptions) - extends SecureConnectionProvider(driver, options) { - override val appEntry: String = { +private[jdbc] class PostgresConnectionProvider extends SecureConnectionProvider { + override val driverClass = "org.postgresql.Driver" + + override def appEntry(driver: Driver, options: JDBCOptions): String = { val parseURL = driver.getClass.getMethod("parseURL", classOf[String], classOf[Properties]) val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties] properties.getProperty("jaasApplicationName", "pgjdbc") } - override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { - val (parent, configEntry) = getConfigWithAppEntry() + override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { + val (parent, configEntry) = getConfigWithAppEntry(driver, options) if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent) + setAuthenticationConfig(parent, driver, options) } } } - -private[sql] object PostgresConnectionProvider { - val driverClass = "org.postgresql.Driver" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala index 24eec63a7244f..80c795957dac8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala @@ -26,39 +26,49 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.util.SecurityUtils -/** - * Some of the secure connection providers modify global JVM security configuration. - * In order to avoid race the modification must be synchronized with this. - */ -private[connection] object SecurityConfigurationLock +private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionProvider with Logging { + /** + * Returns the driver canonical class name which the connection provider supports. + */ + protected val driverClass: String + + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = { + val jdbcOptions = new JDBCOptions(options) + jdbcOptions.keytab != null && jdbcOptions.principal != null && + driverClass.equalsIgnoreCase(jdbcOptions.driverClass) + } -private[jdbc] abstract class SecureConnectionProvider(driver: Driver, options: JDBCOptions) - extends BasicConnectionProvider(driver, options) with Logging { - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - super.getConnection() + override def getConnection(driver: Driver, options: Map[String, String]): Connection = { + val jdbcOptions = new JDBCOptions(options) + setAuthenticationConfigIfNeeded(driver, jdbcOptions) + super.getConnection(driver: Driver, options: Map[String, String]) } /** * Returns JAAS application name. This is sometimes configurable on the JDBC driver level. */ - val appEntry: String + def appEntry(driver: Driver, options: JDBCOptions): String /** * Sets database specific authentication configuration when needed. If configuration already set * then later calls must be no op. When the global JVM security configuration changed then the * related code parts must be synchronized properly. */ - def setAuthenticationConfigIfNeeded(): Unit + def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit - protected def getConfigWithAppEntry(): (Configuration, Array[AppConfigurationEntry]) = { + protected def getConfigWithAppEntry( + driver: Driver, + options: JDBCOptions): (Configuration, Array[AppConfigurationEntry]) = { val parent = Configuration.getConfiguration - (parent, parent.getAppConfigurationEntry(appEntry)) + (parent, parent.getAppConfigurationEntry(appEntry(driver, options))) } - protected def setAuthenticationConfig(parent: Configuration) = { + protected def setAuthenticationConfig( + parent: Configuration, + driver: Driver, + options: JDBCOptions) = { val config = new SecureConnectionProvider.JDBCConfiguration( - parent, appEntry, options.keytab, options.principal) + parent, appEntry(driver, options), options.keytab, options.principal) logDebug("Adding database specific security configuration") Configuration.setConfiguration(config) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala new file mode 100644 index 0000000000000..caf574b0c2284 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcConnectionProvider.scala @@ -0,0 +1,58 @@ +/* + * 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.spark.sql.jdbc + +import java.sql.{Connection, Driver} + +import org.apache.spark.annotation.{DeveloperApi, Unstable} + +/** + * ::DeveloperApi:: + * Connection provider which opens connection toward various databases (database specific instance + * needed). If any authentication required then it's the provider's responsibility to set all + * the parameters. + * Important to mention connection providers within a JVM used from multiple threads so adding + * internal state is not advised. If any state added then it must be synchronized properly. + * + * @since 3.1.0 + */ +@DeveloperApi +@Unstable +abstract class JdbcConnectionProvider { + /** + * Checks if this connection provider instance can handle the connection initiated by the driver. + * There must be exactly one active connection provider which can handle the connection for a + * specific driver. If this requirement doesn't met then `IllegalArgumentException` + * will be thrown by the provider framework. + * + * @param driver Java driver which initiates the connection + * @param options Driver options which initiates the connection + * @return True if the connection provider can handle the driver with the given options. + */ + def canHandle(driver: Driver, options: Map[String, String]): Boolean + + /** + * Opens connection toward the database. Since global JVM security configuration change may needed + * this API is called synchronized by `SecurityConfigurationLock` to avoid race. + * + * @param driver Java driver which initiates the connection + * @param options Driver options which initiates the connection + * @return a `Connection` object that represents a connection to the URL + */ + def getConnection(driver: Driver, options: Map[String, String]): Connection +} diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider new file mode 100644 index 0000000000000..afb48e1a3511f --- /dev/null +++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.jdbc.JdbcConnectionProvider @@ -0,0 +1 @@ +org.apache.spark.sql.execution.datasources.jdbc.connection.IntentionallyFaultyConnectionProvider \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala index ff5fe4f620a1d..a48dbdebea7e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala @@ -20,26 +20,43 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import javax.security.auth.login.Configuration class ConnectionProviderSuite extends ConnectionProviderSuiteBase { + test("All built-in provides must be loaded") { + IntentionallyFaultyConnectionProvider.constructed = false + val providers = ConnectionProvider.loadProviders() + assert(providers.exists(_.isInstanceOf[BasicConnectionProvider])) + assert(providers.exists(_.isInstanceOf[DB2ConnectionProvider])) + assert(providers.exists(_.isInstanceOf[MariaDBConnectionProvider])) + assert(providers.exists(_.isInstanceOf[MSSQLConnectionProvider])) + assert(providers.exists(_.isInstanceOf[PostgresConnectionProvider])) + assert(providers.exists(_.isInstanceOf[OracleConnectionProvider])) + assert(IntentionallyFaultyConnectionProvider.constructed) + assert(!providers.exists(_.isInstanceOf[IntentionallyFaultyConnectionProvider])) + assert(providers.size === 6) + } + test("Multiple security configs must be reachable") { Configuration.setConfiguration(null) - val postgresDriver = registerDriver(PostgresConnectionProvider.driverClass) - val postgresProvider = new PostgresConnectionProvider( - postgresDriver, options("jdbc:postgresql://localhost/postgres")) - val db2Driver = registerDriver(DB2ConnectionProvider.driverClass) - val db2Provider = new DB2ConnectionProvider(db2Driver, options("jdbc:db2://localhost/db2")) + val postgresProvider = new PostgresConnectionProvider() + val postgresDriver = registerDriver(postgresProvider.driverClass) + val postgresOptions = options("jdbc:postgresql://localhost/postgres") + val postgresAppEntry = postgresProvider.appEntry(postgresDriver, postgresOptions) + val db2Provider = new DB2ConnectionProvider() + val db2Driver = registerDriver(db2Provider.driverClass) + val db2Options = options("jdbc:db2://localhost/db2") + val db2AppEntry = db2Provider.appEntry(db2Driver, db2Options) // Make sure no authentication for the databases are set val oldConfig = Configuration.getConfiguration - assert(oldConfig.getAppConfigurationEntry(postgresProvider.appEntry) == null) - assert(oldConfig.getAppConfigurationEntry(db2Provider.appEntry) == null) + assert(oldConfig.getAppConfigurationEntry(postgresAppEntry) == null) + assert(oldConfig.getAppConfigurationEntry(db2AppEntry) == null) - postgresProvider.setAuthenticationConfigIfNeeded() - db2Provider.setAuthenticationConfigIfNeeded() + postgresProvider.setAuthenticationConfigIfNeeded(postgresDriver, postgresOptions) + db2Provider.setAuthenticationConfigIfNeeded(db2Driver, db2Options) // Make sure authentication for the databases are set val newConfig = Configuration.getConfiguration assert(oldConfig != newConfig) - assert(newConfig.getAppConfigurationEntry(postgresProvider.appEntry) != null) - assert(newConfig.getAppConfigurationEntry(db2Provider.appEntry) != null) + assert(newConfig.getAppConfigurationEntry(postgresAppEntry) != null) + assert(newConfig.getAppConfigurationEntry(db2AppEntry) != null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala index d18a3088c4f2f..be08a3c2f7367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala @@ -50,20 +50,25 @@ abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndA } } - protected def testSecureConnectionProvider(provider: SecureConnectionProvider): Unit = { + protected def testSecureConnectionProvider( + provider: SecureConnectionProvider, + driver: Driver, + options: JDBCOptions): Unit = { + val providerAppEntry = provider.appEntry(driver, options) + // Make sure no authentication for the database is set - assert(Configuration.getConfiguration.getAppConfigurationEntry(provider.appEntry) == null) + assert(Configuration.getConfiguration.getAppConfigurationEntry(providerAppEntry) == null) // Make sure the first call sets authentication properly val savedConfig = Configuration.getConfiguration - provider.setAuthenticationConfigIfNeeded() + provider.setAuthenticationConfigIfNeeded(driver, options) val config = Configuration.getConfiguration assert(savedConfig != config) - val appEntry = config.getAppConfigurationEntry(provider.appEntry) + val appEntry = config.getAppConfigurationEntry(providerAppEntry) assert(appEntry != null) // Make sure a second call is not modifying the existing authentication - provider.setAuthenticationConfigIfNeeded() - assert(config.getAppConfigurationEntry(provider.appEntry) === appEntry) + provider.setAuthenticationConfigIfNeeded(driver, options) + assert(config.getAppConfigurationEntry(providerAppEntry) === appEntry) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala index d656f83e2ebb9..5885af82532d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class DB2ConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(DB2ConnectionProvider.driverClass) - val provider = new DB2ConnectionProvider(driver, options("jdbc:db2://localhost/db2")) + val provider = new DB2ConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:db2://localhost/db2")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala new file mode 100644 index 0000000000000..fbefcb91cccde --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/IntentionallyFaultyConnectionProvider.scala @@ -0,0 +1,33 @@ +/* + * 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.spark.sql.execution.datasources.jdbc.connection + +import java.sql.{Connection, Driver} + +import org.apache.spark.sql.jdbc.JdbcConnectionProvider + +private class IntentionallyFaultyConnectionProvider extends JdbcConnectionProvider { + IntentionallyFaultyConnectionProvider.constructed = true + throw new IllegalArgumentException("Intentional Exception") + override def canHandle(driver: Driver, options: Map[String, String]): Boolean = true + override def getConnection(driver: Driver, options: Map[String, String]): Connection = null +} + +private object IntentionallyFaultyConnectionProvider { + var constructed = false +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala index 249f1e36347ed..a5704e842e018 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala @@ -17,35 +17,35 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection +import java.sql.Driver + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") { - val driver = registerDriver(MSSQLConnectionProvider.driverClass) - val defaultProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql")) - val customProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) + val provider = new MSSQLConnectionProvider() + val driver = registerDriver(provider.driverClass) - testProviders(defaultProvider, customProvider) + testProviders(driver, provider, options("jdbc:sqlserver://localhost/mssql"), + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) } test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") { - val parserMethod = "IntentionallyNotExistingMethod" - val driver = registerDriver(MSSQLConnectionProvider.driverClass) - val defaultProvider = new MSSQLConnectionProvider( - driver, options("jdbc:sqlserver://localhost/mssql"), parserMethod) - val customProvider = new MSSQLConnectionProvider( - driver, - options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql"), - parserMethod) - - testProviders(defaultProvider, customProvider) + val provider = new MSSQLConnectionProvider() { + override val parserMethod: String = "IntentionallyNotExistingMethod" + } + val driver = registerDriver(provider.driverClass) + + testProviders(driver, provider, options("jdbc:sqlserver://localhost/mssql"), + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) } private def testProviders( - defaultProvider: SecureConnectionProvider, - customProvider: SecureConnectionProvider) = { - assert(defaultProvider.appEntry !== customProvider.appEntry) - testSecureConnectionProvider(defaultProvider) - testSecureConnectionProvider(customProvider) + driver: Driver, + provider: SecureConnectionProvider, + defaultOptions: JDBCOptions, + customOptions: JDBCOptions) = { + testSecureConnectionProvider(provider, driver, defaultOptions) + testSecureConnectionProvider(provider, driver, customOptions) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala index 70cad2097eb43..f450662fcbe74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class MariaDBConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(MariaDBConnectionProvider.driverClass) - val provider = new MariaDBConnectionProvider(driver, options("jdbc:mysql://localhost/mysql")) + val provider = new MariaDBConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:mysql://localhost/mysql")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala index 13cde32ddbe4e..40e7f1191dccc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class OracleConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(OracleConnectionProvider.driverClass) - val provider = new OracleConnectionProvider(driver, - options("jdbc:oracle:thin:@//localhost/xe")) + val provider = new OracleConnectionProvider() + val driver = registerDriver(provider.driverClass) - testSecureConnectionProvider(provider) + testSecureConnectionProvider(provider, driver, options("jdbc:oracle:thin:@//localhost/xe")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala index 8cef7652f9c54..ee43a7d9708c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala @@ -19,14 +19,14 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class PostgresConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - val driver = registerDriver(PostgresConnectionProvider.driverClass) - val defaultProvider = new PostgresConnectionProvider( - driver, options("jdbc:postgresql://localhost/postgres")) - val customProvider = new PostgresConnectionProvider( - driver, options(s"jdbc:postgresql://localhost/postgres?jaasApplicationName=custompgjdbc")) + val provider = new PostgresConnectionProvider() + val defaultOptions = options("jdbc:postgresql://localhost/postgres") + val customOptions = + options(s"jdbc:postgresql://localhost/postgres?jaasApplicationName=custompgjdbc") + val driver = registerDriver(provider.driverClass) - assert(defaultProvider.appEntry !== customProvider.appEntry) - testSecureConnectionProvider(defaultProvider) - testSecureConnectionProvider(customProvider) + assert(provider.appEntry(driver, defaultOptions) !== provider.appEntry(driver, customOptions)) + testSecureConnectionProvider(provider, driver, defaultOptions) + testSecureConnectionProvider(provider, driver, customOptions) } } From 9996e252ad3ef20760fcbc785e8d3a6f20b6acb5 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 1 Oct 2020 23:01:31 -0700 Subject: [PATCH 20/59] [SPARK-33026][SQL] Add numRows to metric of BroadcastExchangeExec ### What changes were proposed in this pull request? This pr adds `numRows` to the metric and runtimeStatistics of `BroadcastExchangeExec`. ### Why are the changes needed? [`JoinEstimation.estimateInnerOuterJoin`](https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala#L55-L156) need row count. The [ShuffleExchangeExec](https://github.com/apache/spark/blob/1c6dff7b5fc171c190feea0d8f7d323e330d9151/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala#L127) have added the row count, but `BroadcastExchangeExec` missing the row count. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. Closes #29904 from wangyum/SPARK-33026. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../exchange/BroadcastExchangeExec.scala | 5 ++++- .../execution/metric/SQLMetricsSuite.scala | 21 ++++++++++++++++++- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 6d8d37022ea42..4b884dfe537e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -78,6 +78,7 @@ case class BroadcastExchangeExec( override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "numRows" -> SQLMetrics.createMetric(sparkContext, "number of rows"), "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) @@ -90,7 +91,8 @@ case class BroadcastExchangeExec( override def runtimeStatistics: Statistics = { val dataSize = metrics("dataSize").value - Statistics(dataSize) + val numRows = metrics("numRows").value + Statistics(dataSize, Some(numRows)) } @transient @@ -118,6 +120,7 @@ case class BroadcastExchangeExec( throw new SparkException( s"Cannot broadcast the table over $MAX_BROADCAST_TABLE_ROWS rows: $numRows rows") } + longMetric("numRows") += numRows val beforeBuild = System.nanoTime() longMetric("collectTime") += NANOSECONDS.toMillis(beforeBuild - beforeCollect) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 4e10c27edb0e9..e404e460fe611 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -736,4 +736,23 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils Map("dataSize" -> 3200, "shuffleRecordsWritten" -> 100)) testMetricsInSparkPlanOperator(exchanges(1), Map("dataSize" -> 0, "shuffleRecordsWritten" -> 0)) } + + test("Add numRows to metric of BroadcastExchangeExec") { + withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> "true") { + withTable("t1", "t2") { + spark.range(2).write.saveAsTable("t1") + spark.range(2).write.saveAsTable("t2") + val df = sql("SELECT t1.* FROM t1 JOIN t2 ON t1.id = t2.id") + df.collect() + val plan = df.queryExecution.executedPlan + + val exchanges = plan.collect { + case s: BroadcastExchangeExec => s + } + + assert(exchanges.size === 1) + testMetricsInSparkPlanOperator(exchanges.head, Map("numRows" -> 2)) + } + } + } } From b205be5ff6926454b0afe76e4c3438cfa0f34832 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 2 Oct 2020 15:12:33 +0900 Subject: [PATCH 21/59] [SPARK-33051][INFRA][R] Uses setup-r to install R in GitHub Actions build ### What changes were proposed in this pull request? At SPARK-32493, the R installation was switched to manual installation because setup-r was broken. This seems fixed in the upstream so we should better switch it back. ### Why are the changes needed? To avoid maintaining the installation steps by ourselve. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? GitHub Actions build in this PR should test it. Closes #29931 from HyukjinKwon/recover-r-build. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/build_and_test.yml | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 17c040323d515..667371dacf5dc 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -168,12 +168,10 @@ jobs: python3.8 -m pip list # SparkR - name: Install R 4.0 + uses: r-lib/actions/setup-r@v1 if: contains(matrix.modules, 'sparkr') - run: | - sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" - curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add - sudo apt-get update - sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + with: + r-version: 4.0 - name: Install R packages if: contains(matrix.modules, 'sparkr') run: | @@ -232,11 +230,9 @@ jobs: # See also https://github.com/sphinx-doc/sphinx/issues/7551. pip3 install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme ipython nbsphinx - name: Install R 4.0 - run: | - sudo sh -c "echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran40/' >> /etc/apt/sources.list" - curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add - sudo apt-get update - sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + uses: r-lib/actions/setup-r@v1 + with: + r-version: 4.0 - name: Install R linter dependencies and SparkR run: | sudo apt-get install -y libcurl4-openssl-dev From f7ba95264d38484f57c772e459bffb939c9c718e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 2 Oct 2020 15:17:44 +0900 Subject: [PATCH 22/59] [SPARK-33048][BUILD] Fix SparkBuild.scala to recognize build settings for Scala 2.13 ### What changes were proposed in this pull request? This PR fixes `SparkBuild.scala` to recognize build settings for Scala 2.13. In `SparkBuild.scala`, a variable `scalaBinaryVersion` is hardcoded as `2.12`. So, an environment variable `SPARK_SCALA_VERSION` is also to be `2.12`. This issue causes some test suites (e.g. `SparkSubmitSuite`) to be error. ``` ===== TEST OUTPUT FOR o.a.s.deploy.SparkSubmitSuite: 'user classpath first in driver' ===== 20/10/02 08:55:30.234 redirect stderr for command /home/kou/work/oss/spark-scala-2.13/bin/spark-submit INFO Utils: Error: Could not find or load m ain class org.apache.spark.launcher.Main 20/10/02 08:55:30.235 redirect stderr for command /home/kou/work/oss/spark-scala-2.13/bin/spark-submit INFO Utils: /home/kou/work/oss/spark-scala- 2.13/bin/spark-class: line 96: CMD: bad array subscript ``` The reason of this error is that environment variables `SPARK_JARS_DIR` and `LAUNCH_CLASSPATH` is defined in `bin/spark-class` as follows. ``` SPARK_JARS_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION/jars" LAUNCH_CLASSPATH="${SPARK_HOME}/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" ``` ### Why are the changes needed? To build for Scala 2.13 successfully. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests for `core` module finish successfully. ``` build/sbt -Pscala-2.13 clean "core/test" ``` Closes #29927 from sarutak/fix-sparkbuild-for-scala-2.13. Authored-by: Kousuke Saruta Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 28 +--------------------------- 1 file changed, 1 insertion(+), 27 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 160b3b5e7edb3..6328daec027ef 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -94,21 +94,6 @@ object SparkBuild extends PomBuild { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq } - - // TODO: revisit for Scala 2.13 support - /* - Option(System.getProperty("scala.version")) - .filter(_.startsWith("2.11")) - .foreach { versionString => - System.setProperty("scala-2.11", "true") - } - if (System.getProperty("scala-2.11") == "") { - // To activate scala-2.10 profile, replace empty property value to non-empty value - // in the same way as Maven which handles -Dname as -Dname=true before executes build process. - // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082 - System.setProperty("scala-2.11", "true") - } - */ profiles } @@ -965,17 +950,6 @@ object CopyDependencies { object TestSettings { import BuildCommons._ - - // TODO revisit for Scala 2.13 support - private val scalaBinaryVersion = "2.12" - /* - if (System.getProperty("scala-2.11") == "true") { - "2.11" - } else { - "2.12" - } - */ - private val defaultExcludedTags = Seq("org.apache.spark.tags.ChromeUITest") lazy val settings = Seq ( @@ -988,7 +962,7 @@ object TestSettings { (fullClasspath in Test).value.files.map(_.getAbsolutePath) .mkString(File.pathSeparator).stripSuffix(File.pathSeparator), "SPARK_PREPEND_CLASSES" -> "1", - "SPARK_SCALA_VERSION" -> scalaBinaryVersion, + "SPARK_SCALA_VERSION" -> scalaBinaryVersion.value, "SPARK_TESTING" -> "1", "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir", From aa6657981aefae8067672d2c99ca560b6179b723 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 2 Oct 2020 00:06:03 -0700 Subject: [PATCH 23/59] [SPARK-33050][BUILD] Upgrade Apache ORC to 1.5.12 ### What changes were proposed in this pull request? This PR aims to upgrade Apache ORC to 1.5.12. ### Why are the changes needed? This brings us the latest bug patches like the followings. - ORC-644 nested struct evolution does not respect to orc.force.positional.evolution - ORC-667 Positional mapping for nested struct types should not applied by default ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CI. Closes #29930 from dongjoon-hyun/SPARK-ORC-1.5.12. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 6 +++--- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 6 +++--- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 6 +++--- pom.xml | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index fef1a6442cd33..d07b04608328f 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -180,9 +180,9 @@ okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar openshift-model/4.10.3//openshift-model-4.10.3.jar -orc-core/1.5.10/nohive/orc-core-1.5.10-nohive.jar -orc-mapreduce/1.5.10/nohive/orc-mapreduce-1.5.10-nohive.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar +orc-core/1.5.12/nohive/orc-core-1.5.12-nohive.jar +orc-mapreduce/1.5.12/nohive/orc-mapreduce-1.5.12-nohive.jar +orc-shims/1.5.12//orc-shims-1.5.12.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 7b31bdd98ef26..979bb1419ce7b 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -195,9 +195,9 @@ okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar openshift-model/4.10.3//openshift-model-4.10.3.jar -orc-core/1.5.10//orc-core-1.5.10.jar -orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar +orc-core/1.5.12//orc-core-1.5.12.jar +orc-mapreduce/1.5.12//orc-mapreduce-1.5.12.jar +orc-shims/1.5.12//orc-shims-1.5.12.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 960ea5f836ddf..ebaff6d1977c9 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -209,9 +209,9 @@ okhttp/3.12.12//okhttp-3.12.12.jar okio/1.14.0//okio-1.14.0.jar opencsv/2.3//opencsv-2.3.jar openshift-model/4.10.3//openshift-model-4.10.3.jar -orc-core/1.5.10//orc-core-1.5.10.jar -orc-mapreduce/1.5.10//orc-mapreduce-1.5.10.jar -orc-shims/1.5.10//orc-shims-1.5.10.jar +orc-core/1.5.12//orc-core-1.5.12.jar +orc-mapreduce/1.5.12//orc-mapreduce-1.5.12.jar +orc-shims/1.5.12//orc-shims-1.5.12.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index 421d932cef5fa..5d6b0511ce458 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ 2.6.0 10.12.1.1 1.10.1 - 1.5.10 + 1.5.12 com.twitter 1.6.0 From 9b88aca2954cd931c94a7cc788c3c3f7a33e99b7 Mon Sep 17 00:00:00 2001 From: zero323 Date: Fri, 2 Oct 2020 00:53:17 -0700 Subject: [PATCH 24/59] [SPARK-33030][R] Add nth_value to SparkR ### What changes were proposed in this pull request? Adds `nth_value` function to SparkR. ### Why are the changes needed? Feature parity. The function has been already added to [Scala](https://issues.apache.org/jira/browse/SPARK-27951) and [Python](https://issues.apache.org/jira/browse/SPARK-33020). ### Does this PR introduce _any_ user-facing change? Yes. New function is exposed to R users. ### How was this patch tested? New unit tests. Closes #29905 from zero323/SPARK-33030. Authored-by: zero323 Signed-off-by: Dongjoon Hyun --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 34 ++++++++++++++++++++++++++- R/pkg/R/generics.R | 4 ++++ R/pkg/tests/fulltests/test_sparkSQL.R | 2 ++ 4 files changed, 40 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 6d28caff0d56f..4ea05b25ecc9e 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -348,6 +348,7 @@ exportMethods("%<=>%", "negate", "next_day", "not", + "nth_value", "ntile", "otherwise", "over", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index df221de4c7327..18206f6f67778 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -338,7 +338,8 @@ NULL #' tmp <- mutate(df, dist = over(cume_dist(), ws), dense_rank = over(dense_rank(), ws), #' lag = over(lag(df$mpg), ws), lead = over(lead(df$mpg, 1), ws), #' percent_rank = over(percent_rank(), ws), -#' rank = over(rank(), ws), row_number = over(row_number(), ws)) +#' rank = over(rank(), ws), row_number = over(row_number(), ws), +#' nth_value = over(nth_value(df$mpg, 3), ws)) #' # Get ntile group id (1-4) for hp #' tmp <- mutate(tmp, ntile = over(ntile(4), ws)) #' head(tmp)} @@ -3298,6 +3299,37 @@ setMethod("lead", column(jc) }) +#' @details +#' \code{nth_value}: Window function: returns the value that is the \code{offset}th +#' row of the window frame# (counting from 1), and \code{null} if the size of window +#' frame is less than \code{offset} rows. +#' +#' @param offset a numeric indicating number of row to use as the value +#' @param na.rm a logical which indicates that the Nth value should skip null in the +#' determination of which row to use +#' +#' @rdname column_window_functions +#' @aliases nth_value nth_value,characterOrColumn-method +#' @note nth_value since 3.1.0 +setMethod("nth_value", + signature(x = "characterOrColumn", offset = "numeric"), + function(x, offset, na.rm = FALSE) { + x <- if (is.character(x)) { + column(x) + } else { + x + } + offset <- as.integer(offset) + jc <- callJStatic( + "org.apache.spark.sql.functions", + "nth_value", + x@jc, + offset, + na.rm + ) + column(jc) + }) + #' @details #' \code{ntile}: Returns the ntile group id (from 1 to n inclusive) in an ordered window #' partition. For example, if n is 4, the first quarter of the rows will get value 1, the second diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index a7a9379b927b1..985678679dec8 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1164,6 +1164,10 @@ setGeneric("months_between", function(y, x, ...) { standardGeneric("months_betwe #' @rdname count setGeneric("n", function(x) { standardGeneric("n") }) +#' @rdname column_window_functions +#' @name NULL +setGeneric("nth_value", function(x, offset, ...) { standardGeneric("nth_value") }) + #' @rdname column_nonaggregate_functions #' @name NULL setGeneric("nanvl", function(y, x) { standardGeneric("nanvl") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 1c65dabaf6656..c36620227593d 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1425,6 +1425,8 @@ test_that("column functions", { c25 <- overlay(c1, c2, c3, c3) + overlay(c1, c2, c3) + overlay(c1, c2, 1) + overlay(c1, c2, 3, 4) c26 <- timestamp_seconds(c1) + c27 <- nth_value("x", 1L) + nth_value("y", 2, TRUE) + + nth_value(column("v"), 3) + nth_value(column("z"), 4L, FALSE) # Test if base::is.nan() is exposed expect_equal(is.nan(c("a", "b")), c(FALSE, FALSE)) From 82721ce00b6cf535abd3d9cd66445e452554d15d Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 2 Oct 2020 22:16:19 +0900 Subject: [PATCH 25/59] [SPARK-32741][SQL][FOLLOWUP] Run plan integrity check only for effective plan changes ### What changes were proposed in this pull request? (This is a followup PR of #29585) The PR modified `RuleExecutor#isPlanIntegral` code for checking if a plan has globally-unique attribute IDs, but this check made Jenkins maven test jobs much longer (See [the Dongjoon comment](https://github.com/apache/spark/pull/29585#issuecomment-702461314) and thanks, dongjoon-hyun !). To recover running time for the Jenkins tests, this PR intends to update the code to run plan integrity check only for effective plans. ### Why are the changes needed? To recover running time for Jenkins tests. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29928 from maropu/PR29585-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../org/apache/spark/sql/catalyst/rules/RuleExecutor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 3bd8fa78ec92c..d5b0884f6ff13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -229,7 +229,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { tracker.foreach(_.recordRuleInvocation(rule.ruleName, runTime, effective)) // Run the structural integrity checker against the plan after each rule. - if (!isPlanIntegral(result)) { + if (effective && !isPlanIntegral(result)) { val message = s"After applying rule ${rule.ruleName} in batch ${batch.name}, " + "the structural integrity of the plan is broken." throw new TreeNodeException(result, message, null) From 1299c8a81ddba7f0fd8ff1f9afa223a4bb75f7f9 Mon Sep 17 00:00:00 2001 From: Bo Yang Date: Fri, 2 Oct 2020 20:26:46 -0700 Subject: [PATCH 26/59] [SPARK-33037][SHUFFLE] Remove knownManagers to support user's custom shuffle manager plugin ### What changes were proposed in this pull request? Spark has a hardcode list to contain known shuffle managers, which has two values now. It does not contain user's custom shuffle manager which is set through Spark config "spark.shuffle.manager". We hit issue when set "spark.shuffle.manager" with our own shuffle manager plugin (Uber Remote Shuffle Service implementation, https://github.com/uber/RemoteShuffleService). Other users will hit same issue when they implement their own shuffle manager. It is better to remove that knownManagers hardcode list, to support user's custom shuffle manager implementation. ### Why are the changes needed? Spark has shuffle manager API to support custom shuffle manager implementation. The hardcoded known managers list does not consider that shuffle manager config value which could be set by user. Thus better to remove this hardcoded known managers list. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Current Spark unit test already covers the code path. Closes #29916 from boy-uber/knownManagers. Lead-authored-by: Bo Yang Co-authored-by: Bo Yang Signed-off-by: Liang-Chi Hsieh --- .../network/shuffle/ExternalShuffleBlockResolver.java | 8 -------- .../shuffle/ExternalShuffleBlockResolverSuite.java | 9 --------- .../network/shuffle/ExternalShuffleIntegrationSuite.java | 6 +++--- .../scala/org/apache/spark/shuffle/ShuffleManager.scala | 6 +++++- 4 files changed, 8 insertions(+), 21 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index a6bcbb8850566..a095bf2723418 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -92,10 +92,6 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final DB db; - private final List knownManagers = Arrays.asList( - "org.apache.spark.shuffle.sort.SortShuffleManager", - "org.apache.spark.shuffle.unsafe.UnsafeShuffleManager"); - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) throws IOException { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( @@ -148,10 +144,6 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); - if (!knownManagers.contains(executorInfo.shuffleManager)) { - throw new UnsupportedOperationException( - "Unsupported shuffle manager of executor: " + executorInfo); - } try { if (db != null) { byte[] key = dbAppExecKey(fullId); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 88bcf43c2371f..04d4bdf92bae7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -71,15 +71,6 @@ public void testBadRequests() throws IOException { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); } - // Invalid shuffle manager - try { - resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); - resolver.getBlockData("app0", "exec2", 1, 1, 0); - fail("Should have failed"); - } catch (UnsupportedOperationException e) { - // pass - } - // Nonexistent shuffle block resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo(SORT_MANAGER)); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 9d398e372056b..49d02e5dc6fb4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -233,9 +233,9 @@ public void testFetchThreeSort() throws Exception { exec0Fetch.releaseBuffers(); } - @Test (expected = RuntimeException.class) - public void testRegisterInvalidExecutor() throws Exception { - registerExecutor("exec-1", dataContext0.createExecutorInfo("unknown sort manager")); + @Test + public void testRegisterWithCustomShuffleManager() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo("custom shuffle manager")); } @Test diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 400c4526f0114..4e2183451c258 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -24,8 +24,12 @@ import org.apache.spark.{ShuffleDependency, TaskContext} * and on each executor, based on the spark.shuffle.manager setting. The driver registers shuffles * with it, and executors (or tasks running locally in the driver) can ask to read and write data. * - * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and + * NOTE: + * 1. This will be instantiated by SparkEnv so its constructor can take a SparkConf and * boolean isDriver as parameters. + * 2. This contains a method ShuffleBlockResolver which interacts with External Shuffle Service + * when it is enabled. Need to pay attention to that, if implementing a custom ShuffleManager, to + * make sure the custom ShuffleManager could co-exist with External Shuffle Service. */ private[spark] trait ShuffleManager { From 5af62a2ec74356ce1a97c1371321b3424b674289 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 3 Oct 2020 23:37:01 +0900 Subject: [PATCH 27/59] [SPARK-33052][SQL][TEST] Make all the database versions up-to-date for integration tests ### What changes were proposed in this pull request? This PR intends to update database versions below for integration tests; - ibmcom/db2:11.5.0.0a => ibmcom/db2:11.5.4.0 in `DB2[Krb]IntegrationSuite` - mysql:5.7.28 => mysql:5.7.31 in `MySQLIntegrationSuite` - postgres:12.0 => postgres:13.0 in `Postgres[Krb]IntegrationSuite` - mariadb:10.4 => mariadb:10.5 in `MariaDBKrbIntegrationSuite` Also, this added environmental variables so that we can test with any database version and all the variables are as follows (see documents in the code for how to use all the variables); - DB2_DOCKER_IMAGE_NAME - MSSQLSERVER_DOCKER_IMAGE_NAME - MYSQL_DOCKER_IMAGE_NAME - POSTGRES_DOCKER_IMAGE_NAME ### Why are the changes needed? To improve tests. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually checked. Closes #29932 from maropu/UpdateIntegrationTests. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../src/test/resources/mariadb_docker_entrypoint.sh | 2 +- .../apache/spark/sql/jdbc/DB2IntegrationSuite.scala | 9 ++++++++- .../spark/sql/jdbc/DB2KrbIntegrationSuite.scala | 9 ++++++++- .../spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala | 4 +++- .../spark/sql/jdbc/MsSqlServerIntegrationSuite.scala | 10 +++++++++- .../apache/spark/sql/jdbc/MySQLIntegrationSuite.scala | 11 +++++++++-- .../spark/sql/jdbc/PostgresIntegrationSuite.scala | 9 ++++++++- .../spark/sql/jdbc/PostgresKrbIntegrationSuite.scala | 9 ++++++++- 8 files changed, 54 insertions(+), 9 deletions(-) diff --git a/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh index 343bc01651318..97c00a9d81b76 100755 --- a/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh +++ b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh @@ -18,7 +18,7 @@ dpkg-divert --add /bin/systemctl && ln -sT /bin/true /bin/systemctl apt update -apt install -y mariadb-plugin-gssapi-server=1:10.4.12+maria~bionic +apt install -y mariadb-plugin-gssapi-server=1:10.5.5+maria~focal echo "gssapi_keytab_path=/docker-entrypoint-initdb.d/mariadb.keytab" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf echo "gssapi_principal_name=mariadb/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf docker-entrypoint.sh mysqld diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala index 02a7ff8f16073..91498493e78e2 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -25,10 +25,17 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.types.{BooleanType, ByteType, ShortType, StructType} import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.4.0): + * {{{ + * DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.4.0 + * ./build/sbt -Pdocker-integration-tests "test-only *DB2IntegrationSuite" + * }}} + */ @DockerTest class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "ibmcom/db2:11.5.0.0a" + override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.4.0") override val env = Map( "DB2INST1_PASSWORD" -> "rootpass", "LICENSE" -> "accept", diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index fa5ce2d106a10..7ab544c17a5d8 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -29,13 +29,20 @@ import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.sql.execution.datasources.jdbc.connection.{DB2ConnectionProvider, SecureConnectionProvider} import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.4.0): + * {{{ + * DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.4.0 + * ./build/sbt -Pdocker-integration-tests "test-only *DB2KrbIntegrationSuite" + * }}} + */ @DockerTest class DB2KrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val userName = s"db2/$dockerIp" override protected val keytabFileName = "db2.keytab" override val db = new DatabaseOnDocker { - override val imageName = "ibmcom/db2:11.5.0.0a" + override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.4.0") override val env = Map( "DB2INST1_PASSWORD" -> "rootpass", "LICENSE" -> "accept", diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 9b9d15517d572..adee2bebe41ce 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -30,7 +30,9 @@ class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val keytabFileName = "mariadb.keytab" override val db = new DatabaseOnDocker { - override val imageName = "mariadb:10.4" + // If you change `imageName`, you need to update the version of `mariadb-plugin-gssapi-server` + // in `resources/mariadb_docker_entrypoint.sh` accordingly. + override val imageName = "mariadb:10.5" override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 6c633af1fde84..5d3deff9d2704 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -24,10 +24,18 @@ import java.util.Properties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., 2019-GA-ubuntu-16.04): + * {{{ + * MSSQLSERVER_DOCKER_IMAGE_NAME=2019-GA-ubuntu-16.04 + * ./build/sbt -Pdocker-integration-tests "test-only *MsSqlServerIntegrationSuite" + * }}} + */ @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04" + override val imageName = sys.env.getOrElse("MSSQLSERVER_DOCKER_IMAGE_NAME", + "mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04") override val env = Map( "SA_PASSWORD" -> "Sapass123", "ACCEPT_EULA" -> "Y" diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index 4cbcb59e02de1..4cd27f8b9fff2 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -21,13 +21,20 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.Properties -import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.Row import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., mysql:5.7.31): + * {{{ + * MYSQL_DOCKER_IMAGE_NAME=mysql:5.7.31 + * ./build/sbt -Pdocker-integration-tests "test-only *MySQLIntegrationSuite" + * }}} + */ @DockerTest class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mysql:5.7.28" + override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:5.7.31") override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 36d96a69ec659..ba71c942714da 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -26,10 +26,17 @@ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, ShortType} import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., postgres:13.0): + * {{{ + * POSTGRES_DOCKER_IMAGE_NAME=postgres:13.0 + * ./build/sbt -Pdocker-integration-tests "test-only *PostgresIntegrationSuite" + * }}} + */ @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "postgres:12.0-alpine" + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:13.0-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index e94bf3dd588aa..6b215485247d9 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -24,13 +24,20 @@ import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnectionProvider import org.apache.spark.tags.DockerTest +/** + * To run this test suite for a specific version (e.g., postgres:13.0): + * {{{ + * POSTGRES_DOCKER_IMAGE_NAME=postgres:13.0 + * ./build/sbt -Pdocker-integration-tests "test-only *PostgresKrbIntegrationSuite" + * }}} + */ @DockerTest class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val userName = s"postgres/$dockerIp" override protected val keytabFileName = "postgres.keytab" override val db = new DatabaseOnDocker { - override val imageName = "postgres:12.0" + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:13.0") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) From f86171aea43479f54ac2bbbca8f128baa3fc4a8c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 3 Oct 2020 13:12:55 -0500 Subject: [PATCH 28/59] [SPARK-33043][ML] Handle spark.driver.maxResultSize=0 in RowMatrix heuristic computation ### What changes were proposed in this pull request? RowMatrix contains a computation based on spark.driver.maxResultSize. However, when this value is set to 0, the computation fails (log of 0). The fix is simply to correctly handle this setting, which means unlimited result size, by using a tree depth of 1 in the RowMatrix method. ### Why are the changes needed? Simple bug fix to make several Spark ML functions which use RowMatrix run correctly in this case. ### Does this PR introduce _any_ user-facing change? Not other than the bug fix of course. ### How was this patch tested? Existing RowMatrix tests plus a new test. Closes #29925 from srowen/SPARK-33043. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../mllib/linalg/distributed/RowMatrix.scala | 6 +++++- .../mllib/linalg/distributed/RowMatrixSuite.scala | 15 +++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 20e26cee9e0d6..07b9d91c1f59b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -786,11 +786,15 @@ class RowMatrix @Since("1.0.0") ( * Based on the formulae: (numPartitions)^(1/depth) * objectSize <= DriverMaxResultSize * @param aggregatedObjectSizeInBytes the size, in megabytes, of the object being tree aggregated */ - private[spark] def getTreeAggregateIdealDepth(aggregatedObjectSizeInBytes: Long) = { + private[spark] def getTreeAggregateIdealDepth(aggregatedObjectSizeInBytes: Long): Int = { require(aggregatedObjectSizeInBytes > 0, "Cannot compute aggregate depth heuristic based on a zero-size object to aggregate") val maxDriverResultSizeInBytes = rows.conf.get[Long](MAX_RESULT_SIZE) + if (maxDriverResultSizeInBytes <= 0) { + // Unlimited result size, so 1 is OK + return 1 + } require(maxDriverResultSizeInBytes > aggregatedObjectSizeInBytes, s"Cannot aggregate object of size $aggregatedObjectSizeInBytes Bytes, " diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 0a4b11935580a..adc4eeef91bb1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -25,6 +25,7 @@ import breeze.linalg.{norm => brzNorm, svd => brzSvd, DenseMatrix => BDM, DenseV import breeze.numerics.abs import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.MAX_RESULT_SIZE import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors} import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} @@ -121,6 +122,20 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(objectBiggerThanResultSize.getMessage.contains("it's bigger than maxResultSize")) } + test("SPARK-33043: getTreeAggregateIdealDepth with unlimited driver size") { + val originalMaxResultSize = sc.conf.get[Long](MAX_RESULT_SIZE) + sc.conf.set(MAX_RESULT_SIZE, 0L) + try { + val nbPartitions = 100 + val vectors = sc.emptyRDD[Vector] + .repartition(nbPartitions) + val rowMat = new RowMatrix(vectors) + assert(rowMat.getTreeAggregateIdealDepth(700 * 1024 * 1024) === 1) + } finally { + sc.conf.set(MAX_RESULT_SIZE, originalMaxResultSize) + } + } + test("similar columns") { val colMags = Vectors.dense(math.sqrt(126), math.sqrt(66), math.sqrt(94)) val expected = BDM( From 9b21fdd731489b529a52cd2074f79dc7293eed3b Mon Sep 17 00:00:00 2001 From: zero323 Date: Sat, 3 Oct 2020 13:50:38 -0700 Subject: [PATCH 29/59] [SPARK-32949][FOLLOW-UP][R][SQL] Reindent lines in SparkR timestamp_seconds ### What changes were proposed in this pull request? Re-indent lines of SparkR `timestamp_seconds`. ### Why are the changes needed? Current indentation is not aligned with the opening line. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29940 from zero323/SPARK-32949-FOLLOW-UP. Authored-by: zero323 Signed-off-by: Dongjoon Hyun --- R/pkg/R/functions.R | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 18206f6f67778..b216f404a3ca5 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -4451,10 +4451,10 @@ setMethod("current_timestamp", #' @aliases timestamp_seconds timestamp_seconds,Column-method #' @note timestamp_seconds since 3.1.0 setMethod("timestamp_seconds", - signature(x = "Column"), - function(x) { - jc <- callJStatic( - "org.apache.spark.sql.functions", "timestamp_seconds", x@jc - ) - column(jc) - }) + signature(x = "Column"), + function(x) { + jc <- callJStatic( + "org.apache.spark.sql.functions", "timestamp_seconds", x@jc + ) + column(jc) + }) From 37c806af2bd3fb4c1f25e02f4986226e5e8d994d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 3 Oct 2020 14:55:02 -0700 Subject: [PATCH 30/59] [SPARK-32958][SQL] Prune unnecessary columns from JsonToStructs ### What changes were proposed in this pull request? This patch proposes to do column pruning for `JsonToStructs` expression if we only require some fields from it. ### Why are the changes needed? `JsonToStructs` takes a schema parameter used to tell `JacksonParser` what fields are needed to parse. If `JsonToStructs` is followed by `GetStructField`. We can prune the schema to only parse certain field. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test Closes #29900 from viirya/SPARK-32958. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../optimizer/OptimizeJsonExprs.scala | 16 +++++ .../optimizer/OptimizeJsonExprsSuite.scala | 58 +++++++++++++++++++ 2 files changed, 74 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala index 24df480208220..59228904d84b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala @@ -20,9 +20,14 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types.{ArrayType, StructType} /** * Simplify redundant json related expressions. + * + * The optimization includes: + * 1. JsonToStructs(StructsToJson(child)) => child. + * 2. Prune unnecessary columns from GetStructField/GetArrayStructFields + JsonToStructs. */ object OptimizeJsonExprs extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -38,6 +43,17 @@ object OptimizeJsonExprs extends Rule[LogicalPlan] { // so `JsonToStructs` might throw error in runtime. Thus we cannot optimize // this case similarly. child + + case g @ GetStructField(j @ JsonToStructs(schema: StructType, _, _, _), ordinal, _) + if schema.length > 1 => + val prunedSchema = StructType(Seq(schema(ordinal))) + g.copy(child = j.copy(schema = prunedSchema), ordinal = 0) + + case g @ GetArrayStructFields(j @ JsonToStructs(schema: ArrayType, _, _, _), _, _, _, _) + if schema.elementType.asInstanceOf[StructType].length > 1 => + val prunedSchema = ArrayType(StructType(Seq(g.field)), g.containsNull) + g.copy(child = j.copy(schema = prunedSchema), ordinal = 0, numFields = 1) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala index 90397d4cabee8..e47a141dfed1f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala @@ -36,8 +36,10 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val schema = StructType.fromDDL("a int, b int") private val structAtt = 'struct.struct(schema).notNull + private val jsonAttr = 'json.string private val testRelation = LocalRelation(structAtt) + private val testRelation2 = LocalRelation(jsonAttr) test("SPARK-32948: optimize from_json + to_json") { val options = Map.empty[String, String] @@ -141,4 +143,60 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val expected2 = testRelation.select('struct.as("struct")).analyze comparePlans(optimized2, expected2) } + + test("SPARK-32958: prune unnecessary columns from GetStructField + from_json") { + val options = Map.empty[String, String] + + val query1 = testRelation2 + .select(GetStructField(JsonToStructs(schema, options, 'json), 0)) + val optimized1 = Optimizer.execute(query1.analyze) + + val prunedSchema1 = StructType.fromDDL("a int") + val expected1 = testRelation2 + .select(GetStructField(JsonToStructs(prunedSchema1, options, 'json), 0)).analyze + comparePlans(optimized1, expected1) + + val query2 = testRelation2 + .select(GetStructField(JsonToStructs(schema, options, 'json), 1)) + val optimized2 = Optimizer.execute(query2.analyze) + + val prunedSchema2 = StructType.fromDDL("b int") + val expected2 = testRelation2 + .select(GetStructField(JsonToStructs(prunedSchema2, options, 'json), 0)).analyze + comparePlans(optimized2, expected2) + } + + test("SPARK-32958: prune unnecessary columns from GetArrayStructFields + from_json") { + val options = Map.empty[String, String] + val schema1 = ArrayType(StructType.fromDDL("a int, b int"), containsNull = true) + val field1 = schema1.elementType.asInstanceOf[StructType](0) + + val query1 = testRelation2 + .select(GetArrayStructFields( + JsonToStructs(schema1, options, 'json), field1, 0, 2, true).as("a")) + val optimized1 = Optimizer.execute(query1.analyze) + + val prunedSchema1 = ArrayType(StructType.fromDDL("a int"), containsNull = true) + val expected1 = testRelation2 + .select(GetArrayStructFields( + JsonToStructs(prunedSchema1, options, 'json), field1, 0, 1, true).as("a")).analyze + comparePlans(optimized1, expected1) + + val schema2 = ArrayType( + StructType( + StructField("a", IntegerType, false) :: + StructField("b", IntegerType, false) :: Nil), containsNull = false) + val field2 = schema2.elementType.asInstanceOf[StructType](1) + val query2 = testRelation2 + .select(GetArrayStructFields( + JsonToStructs(schema2, options, 'json), field2, 1, 2, false).as("b")) + val optimized2 = Optimizer.execute(query2.analyze) + + val prunedSchema2 = ArrayType( + StructType(StructField("b", IntegerType, false) :: Nil), containsNull = false) + val expected2 = testRelation2 + .select(GetArrayStructFields( + JsonToStructs(prunedSchema2, options, 'json), field2, 0, 1, false).as("b")).analyze + comparePlans(optimized2, expected2) + } } From db420f79cc588dc0f98b906accb34d63a1e4664c Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 3 Oct 2020 15:14:48 -0700 Subject: [PATCH 31/59] [SPARK-33049][CORE] Decommission shuffle block test is flaky ### What changes were proposed in this pull request? Increase the listener bus event length, syncrhonize the addition of blocks modified to the array list. ### Why are the changes needed? This test appears flaky in Jenkins (can not repro locally). Given that the index file made it through and the index file is only transferred after the data file, the only two reasons I could come up with an interminentent failure here are with the listenerbus dropping a message or the two block change messages being received at the same time. ### Does this PR introduce _any_ user-facing change? No (test only). ### How was this patch tested? The tests still pass on my machine but they did before. We'll need to run it through jenkins a few times first. Closes #29929 from holdenk/fix-.BlockManagerDecommissionIntegrationSuite. Authored-by: Holden Karau Signed-off-by: Dongjoon Hyun --- .../storage/BlockManagerDecommissionIntegrationSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 094b893cdda2e..dcf313f671d5e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -69,6 +69,8 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS .set(config.STORAGE_DECOMMISSION_ENABLED, true) .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist) .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle) + // Since we use the bus for testing we don't want to drop any messages + .set(config.LISTENER_BUS_EVENT_QUEUE_CAPACITY, 1000000) // Just replicate blocks quickly during testing, there isn't another // workload we need to worry about. .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 10L) @@ -137,7 +139,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS taskEndEvents.add(taskEnd) } - override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = synchronized { blocksUpdated.append(blockUpdated) } From fab53212cb110a81696cee8546c35095332f6e09 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 4 Oct 2020 16:11:06 -0700 Subject: [PATCH 32/59] [SPARK-33065][TESTS] Expand the stack size of a thread in a test in LocalityPlacementStrategySuite for Java 11 with sbt ### What changes were proposed in this pull request? This PR fixes an issue that a test in `LocalityPlacementStrategySuite` fails with Java 11 due to `StackOverflowError`. ``` [info] - handle large number of containers and tasks (SPARK-18750) *** FAILED *** (170 milliseconds) [info] StackOverflowError should not be thrown; however, got: [info] [info] java.lang.StackOverflowError [info] at java.base/java.util.concurrent.ConcurrentHashMap.putVal(ConcurrentHashMap.java:1012) [info] at java.base/java.util.concurrent.ConcurrentHashMap.putIfAbsent(ConcurrentHashMap.java:1541) [info] at java.base/java.lang.ClassLoader.getClassLoadingLock(ClassLoader.java:668) [info] at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:591) [info] at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579) [info] at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178) [info] at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522) ``` The solution is to expand the stack size of a thread in the test from 32KB to 256KB. Currently, the stack size is specified as 32KB but the actual stack size can be greater than 32KB. According to the code of Hotspot, the minimum stack size is prefer to the specified size. Java 8: https://hg.openjdk.java.net/jdk8u/jdk8u/hotspot/file/c92ba514724d/src/os/linux/vm/os_linux.cpp#l900 Java 11: https://hg.openjdk.java.net/jdk-updates/jdk11u/file/73edf743a93a/src/hotspot/os/posix/os_posix.cpp#l1555 For Linux on x86_64, the minimum stack size seems to be 224KB and 136KB for Java 8 and Java 11 respectively. So, the actual stack size should be 224KB rather than 32KB for Java 8 on x86_64/Linux. As the test passes for Java 8 but doesn't for Java 11, 224KB is enough while 136KB is not. So I think specifing 256KB is reasonable for the new stack size. ### Why are the changes needed? To pass the test for Java 11. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Following command with Java 11. ``` build/sbt -Pyarn clean package "testOnly org.apache.spark.deploy.yarn.LocalityPlacementStrategySuite" ``` Closes #29943 from sarutak/fix-stack-size. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../spark/deploy/yarn/LocalityPlacementStrategySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala index 3c9209c292418..d2397504ba140 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala @@ -43,7 +43,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite { } } - val thread = new Thread(new ThreadGroup("test"), runnable, "test-thread", 32 * 1024) + val thread = new Thread(new ThreadGroup("test"), runnable, "test-thread", 256 * 1024) thread.start() thread.join() From 4ab9aa03055d3ad90137efacb2e00eff4ac3fbf1 Mon Sep 17 00:00:00 2001 From: reidy-p Date: Mon, 5 Oct 2020 11:48:28 +0900 Subject: [PATCH 33/59] [SPARK-33017][PYTHON] Add getCheckpointDir method to PySpark Context ### What changes were proposed in this pull request? Adding a method to get the checkpoint directory from the PySpark context to match the Scala API ### Why are the changes needed? To make the Scala and Python APIs consistent and remove the need to use the JavaObject ### Does this PR introduce _any_ user-facing change? Yes, there is a new method which makes it easier to get the checkpoint directory directly rather than using the JavaObject #### Previous behaviour: ```python >>> spark.sparkContext.setCheckpointDir('/tmp/spark/checkpoint/') >>> sc._jsc.sc().getCheckpointDir().get() 'file:/tmp/spark/checkpoint/63f7b67c-e5dc-4d11-a70c-33554a71717a' ``` This method returns a confusing Scala error if it has not been set ```python >>> sc._jsc.sc().getCheckpointDir().get() Traceback (most recent call last): File "", line 1, in File "/home/paul/Desktop/spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__ File "/home/paul/Desktop/spark/python/pyspark/sql/utils.py", line 111, in deco return f(*a, **kw) File "/home/paul/Desktop/spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 328, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o25.get. : java.util.NoSuchElementException: None.get at scala.None$.get(Option.scala:529) at scala.None$.get(Option.scala:527) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:282) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` #### New method: ```python >>> spark.sparkContext.setCheckpointDir('/tmp/spark/checkpoint/') >>> spark.sparkContext.getCheckpointDir() 'file:/tmp/spark/checkpoint/b38aca2e-8ace-44fc-a4c4-f4e36c2da2a7' ``` ``getCheckpointDir()`` returns ``None`` if it has not been set ```python >>> print(spark.sparkContext.getCheckpointDir()) None ``` ### How was this patch tested? Added to existing unit tests. But I'm not sure how to add a test for the case where ``getCheckpointDir()`` should return ``None`` since the existing checkpoint tests set the checkpoint directory in the ``setUp`` method before any tests are run as far as I can tell. Closes #29918 from reidy-p/SPARK-33017. Authored-by: reidy-p Signed-off-by: HyukjinKwon --- python/pyspark/__init__.py | 3 ++- python/pyspark/context.py | 12 +++++++++++- python/pyspark/context.pyi | 1 + python/pyspark/tests/test_context.py | 3 +++ 4 files changed, 17 insertions(+), 2 deletions(-) diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index fb05819e74124..19269e4466507 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -50,7 +50,6 @@ import types from pyspark.conf import SparkConf -from pyspark.context import SparkContext from pyspark.rdd import RDD, RDDBarrier from pyspark.files import SparkFiles from pyspark.status import StatusTracker, SparkJobInfo, SparkStageInfo @@ -113,6 +112,8 @@ def wrapper(self, *args, **kwargs): return func(self, **kwargs) return wrapper +# To avoid circular dependencies +from pyspark.context import SparkContext # for back compatibility from pyspark.sql import SQLContext, HiveContext, Row # noqa: F401 diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 55a5657b64055..4213a742a1dc4 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -28,7 +28,7 @@ from py4j.protocol import Py4JError from py4j.java_gateway import is_instance_of -from pyspark import accumulators +from pyspark import accumulators, since from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast, BroadcastPickleRegistry from pyspark.conf import SparkConf @@ -956,6 +956,16 @@ def setCheckpointDir(self, dirName): """ self._jsc.sc().setCheckpointDir(dirName) + @since(3.1) + def getCheckpointDir(self): + """ + Return the directory where RDDs are checkpointed. Returns None if no + checkpoint directory has been set. + """ + if not self._jsc.sc().getCheckpointDir().isEmpty(): + return self._jsc.sc().getCheckpointDir().get() + return None + def _getJavaStorageLevel(self, storageLevel): """ Returns a Java StorageLevel based on a pyspark.StorageLevel. diff --git a/python/pyspark/context.pyi b/python/pyspark/context.pyi index 76ecf8911471a..2789a38b3be9f 100644 --- a/python/pyspark/context.pyi +++ b/python/pyspark/context.pyi @@ -152,6 +152,7 @@ class SparkContext: def addFile(self, path: str, recursive: bool = ...) -> None: ... def addPyFile(self, path: str) -> None: ... def setCheckpointDir(self, dirName: str) -> None: ... + def getCheckpointDir(self) -> Optional[str]: ... def setJobGroup( self, groupId: str, description: str, interruptOnCancel: bool = ... ) -> None: ... diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 9b6b74a111288..d86f6c3c1571c 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -43,6 +43,7 @@ def test_basic_checkpointing(self): self.assertFalse(flatMappedRDD.isCheckpointed()) self.assertTrue(flatMappedRDD.getCheckpointFile() is None) + self.assertFalse(self.sc.getCheckpointDir() is None) flatMappedRDD.checkpoint() result = flatMappedRDD.collect() @@ -51,6 +52,8 @@ def test_basic_checkpointing(self): self.assertEqual(flatMappedRDD.collect(), result) self.assertEqual("file:" + self.checkpointDir.name, os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile()))) + self.assertEqual(self.sc.getCheckpointDir(), + os.path.dirname(flatMappedRDD.getCheckpointFile())) def test_checkpoint_and_restore(self): parCollection = self.sc.parallelize([1, 2, 3, 4]) From e83d03ca4861a69cd688beacc544b3f6dae32ae0 Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 5 Oct 2020 13:18:12 +0900 Subject: [PATCH 34/59] [SPARK-33040][R][ML] Add SparkR wrapper for vector_to_array ### What changes were proposed in this pull request? Add SparkR wrapper for `o.a.s.ml.functions.vector_to_array` ### Why are the changes needed? - Currently ML vectors, including predictions, are almost inaccessible to R users. That's is a serious loss of functionality. - Feature parity. ### Does this PR introduce _any_ user-facing change? Yes, new R function is added. ### How was this patch tested? - New unit tests. - Manual verification. Closes #29917 from zero323/SPARK-33040. Authored-by: zero323 Signed-off-by: HyukjinKwon --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 33 +++++++++++++++++++++++++++ R/pkg/R/generics.R | 4 ++++ R/pkg/tests/fulltests/test_sparkSQL.R | 3 ++- 4 files changed, 40 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 4ea05b25ecc9e..25162f3e23b38 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -427,6 +427,7 @@ exportMethods("%<=>%", "variance", "var_pop", "var_samp", + "vector_to_array", "weekofyear", "when", "window", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index b216f404a3ca5..61ea90efb348d 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -345,6 +345,17 @@ NULL #' head(tmp)} NULL +#' ML functions for Column operations +#' +#' ML functions defined for \code{Column}. +#' +#' @param x Column to compute on. +#' @param ... additional argument(s). +#' @name column_ml_functions +#' @rdname column_ml_functions +#' @family ml functions +NULL + #' @details #' \code{lit}: A new Column is created to represent the literal value. #' If the parameter is a Column, it is returned unchanged. @@ -4458,3 +4469,25 @@ setMethod("timestamp_seconds", ) column(jc) }) + +#' @details +#' \code{vector_to_array} Converts a column of MLlib sparse/dense vectors into +#' a column of dense arrays. +#' +#' @param dtype The data type of the output array. Valid values: "float64" or "float32". +#' +#' @rdname column_ml_functions +#' @aliases vector_to_array vector_to_array,Column-method +#' @note vector_to_array since 3.1.0 +setMethod("vector_to_array", + signature(x = "Column"), + function(x, dtype = c("float32", "float64")) { + dtype <- match.arg(dtype) + jc <- callJStatic( + "org.apache.spark.ml.functions", + "vector_to_array", + x@jc, + dtype + ) + column(jc) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 985678679dec8..993fc758adbe5 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1449,6 +1449,10 @@ setGeneric("var_pop", function(x) { standardGeneric("var_pop") }) #' @name NULL setGeneric("var_samp", function(x) { standardGeneric("var_samp") }) +#' @rdname column_ml_functions +#' @name NULL +setGeneric("vector_to_array", function(x, ...) { standardGeneric("vector_to_array") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index c36620227593d..c3b271b1205c5 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1424,7 +1424,8 @@ test_that("column functions", { date_trunc("quarter", c) + current_date() + current_timestamp() c25 <- overlay(c1, c2, c3, c3) + overlay(c1, c2, c3) + overlay(c1, c2, 1) + overlay(c1, c2, 3, 4) - c26 <- timestamp_seconds(c1) + c26 <- timestamp_seconds(c1) + vector_to_array(c) + + vector_to_array(c, "float32") + vector_to_array(c, "float64") c27 <- nth_value("x", 1L) + nth_value("y", 2, TRUE) + nth_value(column("v"), 3) + nth_value(column("z"), 4L, FALSE) From 24f890e8e81ee03fe0d9ce4c8f232784e9fdaccd Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 5 Oct 2020 16:31:17 +0900 Subject: [PATCH 35/59] [SPARK-33040][FOLLOW-UP][R] Reorder argument choices and add examples ### What changes were proposed in this pull request? - Reorder choices of `dtype` to match Scala defaults. - Add example to ml_functions. ### Why are the changes needed? As requested: - https://github.com/apache/spark/pull/29917#pullrequestreview-501715344 - https://github.com/apache/spark/pull/29917#pullrequestreview-501716521 ### Does this PR introduce _any_ user-facing change? No (changes to newly added component). ### How was this patch tested? Existing tests. Closes #29944 from zero323/SPARK-33040-FOLLOW-UP. Authored-by: zero323 Signed-off-by: HyukjinKwon --- R/pkg/R/functions.R | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 61ea90efb348d..959edf29e2429 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -354,6 +354,11 @@ NULL #' @name column_ml_functions #' @rdname column_ml_functions #' @family ml functions +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +#' head(select(df, vector_to_array(df$features))) +#' } NULL #' @details @@ -4481,7 +4486,7 @@ setMethod("timestamp_seconds", #' @note vector_to_array since 3.1.0 setMethod("vector_to_array", signature(x = "Column"), - function(x, dtype = c("float32", "float64")) { + function(x, dtype = c("float64", "float32")) { dtype <- match.arg(dtype) jc <- callJStatic( "org.apache.spark.ml.functions", From 0fb2574d4e75fa4a545da1d53357c2359c0bffeb Mon Sep 17 00:00:00 2001 From: Yuning Zhang Date: Mon, 5 Oct 2020 20:25:57 +0900 Subject: [PATCH 36/59] [SPARK-33042][SQL][TEST] Add a test case to ensure changes to spark.sql.optimizer.maxIterations take effect at runtime ### What changes were proposed in this pull request? Add a test case to ensure changes to `spark.sql.optimizer.maxIterations` take effect at runtime. ### Why are the changes needed? Currently, there is only one related test case: https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala#L156 However, this test case only checks the value of the conf can be changed at runtime. It does not check the updated value is actually used by the Optimizer. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? unit test Closes #29919 from yuningzh-db/add_optimizer_test. Authored-by: Yuning Zhang Signed-off-by: HyukjinKwon --- .../catalyst/optimizer/OptimizerSuite.scala | 74 +++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala new file mode 100644 index 0000000000000..b48555ec2fb28 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala @@ -0,0 +1,74 @@ +/* + * 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.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions.{Alias, IntegerLiteral, Literal} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf + +/** + * A dummy optimizer rule for testing that decrements integer literals until 0. + */ +object DecrementLiterals extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformExpressions { + case IntegerLiteral(i) if i > 0 => Literal(i - 1) + } +} + +class OptimizerSuite extends PlanTest { + test("Optimizer exceeds max iterations") { + val iterations = 5 + val maxIterationsNotEnough = 3 + val maxIterationsEnough = 10 + val analyzed = Project(Alias(Literal(iterations), "attr")() :: Nil, OneRowRelation()).analyze + + withSQLConf(SQLConf.OPTIMIZER_MAX_ITERATIONS.key -> maxIterationsNotEnough.toString) { + val optimizer = new SimpleTestOptimizer() { + override def defaultBatches: Seq[Batch] = + Batch("test", fixedPoint, + DecrementLiterals) :: Nil + } + + val message1 = intercept[TreeNodeException[LogicalPlan]] { + optimizer.execute(analyzed) + }.getMessage + assert(message1.startsWith(s"Max iterations ($maxIterationsNotEnough) reached for batch " + + s"test, please set '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}' to a larger value.")) + + withSQLConf(SQLConf.OPTIMIZER_MAX_ITERATIONS.key -> maxIterationsEnough.toString) { + try { + optimizer.execute(analyzed) + } catch { + case ex: TreeNodeException[LogicalPlan] + if ex.getMessage.contains(SQLConf.OPTIMIZER_MAX_ITERATIONS.key) => + fail("optimizer.execute should not reach max iterations.") + } + } + + val message2 = intercept[TreeNodeException[LogicalPlan]] { + optimizer.execute(analyzed) + }.getMessage + assert(message2.startsWith(s"Max iterations ($maxIterationsNotEnough) reached for batch " + + s"test, please set '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}' to a larger value.")) + } + } +} From 023eb482b23b5d63d2157b3def9926673844e0a3 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 5 Oct 2020 22:00:42 +0900 Subject: [PATCH 37/59] [SPARK-32914][SQL] Avoid constructing dataType multiple times ### What changes were proposed in this pull request? Some expression's data type not a static value. It needs to be constructed a new object when calling `dataType` function. E.g.: `CaseWhen`. We should avoid constructing dataType multiple times because it may be used many times. E.g.: [`HyperLogLogPlusPlus.update`](https://github.com/apache/spark/blob/10edeafc69250afef8c71ed7b3c77992f67aa4ff/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala#L122). ### Why are the changes needed? Improve query performance. for example: ```scala spark.range(100000000L).selectExpr("approx_count_distinct(case when id % 400 > 20 then id else 0 end)").show ``` Profiling result: ``` -- Execution profile --- Total samples : 18365 Frame buffer usage : 2.6688% --- 58443254327 ns (31.82%), 5844 samples [ 0] GenericTaskQueueSet, (MemoryType)1>::steal_best_of_2(unsigned int, int*, StarTask&) [ 1] StealTask::do_it(GCTaskManager*, unsigned int) [ 2] GCTaskThread::run() [ 3] java_start(Thread*) [ 4] start_thread --- 6140668667 ns (3.34%), 614 samples [ 0] GenericTaskQueueSet, (MemoryType)1>::peek() [ 1] ParallelTaskTerminator::offer_termination(TerminatorTerminator*) [ 2] StealTask::do_it(GCTaskManager*, unsigned int) [ 3] GCTaskThread::run() [ 4] java_start(Thread*) [ 5] start_thread --- 5679994036 ns (3.09%), 568 samples [ 0] scala.collection.generic.Growable.$plus$plus$eq [ 1] scala.collection.generic.Growable.$plus$plus$eq$ [ 2] scala.collection.mutable.ListBuffer.$plus$plus$eq [ 3] scala.collection.mutable.ListBuffer.$plus$plus$eq [ 4] scala.collection.generic.GenericTraversableTemplate.$anonfun$flatten$1 [ 5] scala.collection.generic.GenericTraversableTemplate$$Lambda$107.411506101.apply [ 6] scala.collection.immutable.List.foreach [ 7] scala.collection.generic.GenericTraversableTemplate.flatten [ 8] scala.collection.generic.GenericTraversableTemplate.flatten$ [ 9] scala.collection.AbstractTraversable.flatten [10] org.apache.spark.internal.config.ConfigEntry.readString [11] org.apache.spark.internal.config.ConfigEntryWithDefault.readFrom [12] org.apache.spark.sql.internal.SQLConf.getConf [13] org.apache.spark.sql.internal.SQLConf.caseSensitiveAnalysis [14] org.apache.spark.sql.types.DataType.sameType [15] org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1 [16] org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1$adapted [17] org.apache.spark.sql.catalyst.analysis.TypeCoercion$$$Lambda$1527.1975399904.apply [18] scala.collection.IndexedSeqOptimized.prefixLengthImpl [19] scala.collection.IndexedSeqOptimized.forall [20] scala.collection.IndexedSeqOptimized.forall$ [21] scala.collection.mutable.ArrayBuffer.forall [22] org.apache.spark.sql.catalyst.analysis.TypeCoercion$.haveSameType [23] org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck [24] org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck$ [25] org.apache.spark.sql.catalyst.expressions.CaseWhen.dataTypeCheck [26] org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType [27] org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType$ [28] org.apache.spark.sql.catalyst.expressions.CaseWhen.dataType [29] org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlus.update [30] org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1.$anonfun$applyOrElse$2 [31] org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1.$anonfun$applyOrElse$2$adapted [32] org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$Lambda$1534.1383512673.apply [33] org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateProcessRow$7 [34] org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateProcessRow$7$adapted [35] org.apache.spark.sql.execution.aggregate.AggregationIterator$$Lambda$1555.725788712.apply ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test and benchmark test: Benchmark code | Before this PR(Milliseconds) | After this PR(Milliseconds) --- | --- | --- spark.range(100000000L).selectExpr("approx_count_distinct(case when id % 400 > 20 then id else 0 end)").collect() | 56462 | 3794 Closes #29790 from wangyum/SPARK-32914. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/expressions/Expression.scala | 4 +++- .../aggregate/ApproximatePercentile.scala | 4 +++- .../catalyst/expressions/collectionOperations.scala | 13 ++++++++++--- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ce4aa1c2b7c2f..35b192cc5544a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1060,10 +1060,12 @@ trait ComplexTypeMergingExpression extends Expression { s" The input types found are\n\t${inputTypesForMerging.mkString("\n\t")}") } - override def dataType: DataType = { + private lazy val internalDataType: DataType = { dataTypeCheck inputTypesForMerging.reduceLeft(TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(_, _).get) } + + override def dataType: DataType = internalDataType } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 3327f4ccf4461..2a5275e75d4f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -187,10 +187,12 @@ case class ApproximatePercentile( override def nullable: Boolean = true // The result type is the same as the input type. - override def dataType: DataType = { + private lazy val internalDataType: DataType = { if (returnPercentileArray) ArrayType(child.dataType, false) else child.dataType } + override def dataType: DataType = internalDataType + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("percentile_approx") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 8555f63df986f..8719b2e065663 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -371,7 +371,7 @@ case class MapEntries(child: Expression) @transient private lazy val childDataType: MapType = child.dataType.asInstanceOf[MapType] - override def dataType: DataType = { + private lazy val internalDataType: DataType = { ArrayType( StructType( StructField("key", childDataType.keyType, false) :: @@ -380,6 +380,8 @@ case class MapEntries(child: Expression) false) } + override def dataType: DataType = internalDataType + override protected def nullSafeEval(input: Any): Any = { val childMap = input.asInstanceOf[MapData] val keys = childMap.keyArray() @@ -3504,13 +3506,16 @@ object ArrayUnion { since = "2.4.0") case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBinaryLike with ComplexTypeMergingExpression { - override def dataType: DataType = { + + private lazy val internalDataType: DataType = { dataTypeCheck ArrayType(elementType, left.dataType.asInstanceOf[ArrayType].containsNull && right.dataType.asInstanceOf[ArrayType].containsNull) } + override def dataType: DataType = internalDataType + @transient lazy val evalIntersect: (ArrayData, ArrayData) => ArrayData = { if (TypeUtils.typeWithProperEquals(elementType)) { (array1, array2) => @@ -3747,11 +3752,13 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryLike with ComplexTypeMergingExpression { - override def dataType: DataType = { + private lazy val internalDataType: DataType = { dataTypeCheck left.dataType } + override def dataType: DataType = internalDataType + @transient lazy val evalExcept: (ArrayData, ArrayData) => ArrayData = { if (TypeUtils.typeWithProperEquals(elementType)) { (array1, array2) => From a09747bf326677e212fbc284285cce822571c315 Mon Sep 17 00:00:00 2001 From: gschiavon Date: Mon, 5 Oct 2020 09:02:06 -0700 Subject: [PATCH 38/59] [SPARK-33063][K8S] Improve error message for insufficient K8s volume confs ### What changes were proposed in this pull request? Provide error handling when creating kubernetes volumes. Right now they keys are expected to be there and if not it fails with a `key not found` error, but not knowing why do you need that `key`. Also I renamed some tests that didn't indicate the kind of kubernetes volume ### Why are the changes needed? Easier for the users to understand why `spark-submit` command is failing if not providing they right kubernetes volumes properties. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? It was tested with the current tests plus added one more. [Jira ticket](https://issues.apache.org/jira/browse/SPARK-33063) Closes #29941 from Gschiavon/SPARK-33063-provide-error-handling-k8s-volumes. Authored-by: gschiavon Signed-off-by: Dongjoon Hyun --- .../spark/deploy/k8s/KubernetesVolumeUtils.scala | 10 ++++++++++ .../spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala | 11 +++++++++++ 2 files changed, 21 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala index 77921f6338c74..b2eacca042794 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala @@ -67,6 +67,7 @@ private[spark] object KubernetesVolumeUtils { volumeType match { case KUBERNETES_VOLUMES_HOSTPATH_TYPE => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_PATH_KEY" + verifyOptionKey(options, pathKey, KUBERNETES_VOLUMES_HOSTPATH_TYPE) KubernetesHostPathVolumeConf(options(pathKey)) case KUBERNETES_VOLUMES_PVC_TYPE => @@ -74,6 +75,7 @@ private[spark] object KubernetesVolumeUtils { val storageClassKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_CLAIM_STORAGE_CLASS_KEY" val sizeLimitKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY" + verifyOptionKey(options, claimNameKey, KUBERNETES_VOLUMES_PVC_TYPE) KubernetesPVCVolumeConf( options(claimNameKey), options.get(storageClassKey), @@ -87,6 +89,8 @@ private[spark] object KubernetesVolumeUtils { case KUBERNETES_VOLUMES_NFS_TYPE => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_PATH_KEY" val serverKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_SERVER_KEY" + verifyOptionKey(options, pathKey, KUBERNETES_VOLUMES_NFS_TYPE) + verifyOptionKey(options, serverKey, KUBERNETES_VOLUMES_NFS_TYPE) KubernetesNFSVolumeConf( options(pathKey), options(serverKey)) @@ -95,4 +99,10 @@ private[spark] object KubernetesVolumeUtils { throw new IllegalArgumentException(s"Kubernetes Volume type `$volumeType` is not supported") } } + + private def verifyOptionKey(options: Map[String, String], key: String, msg: String): Unit = { + if (!options.isDefinedAt(key)) { + throw new NoSuchElementException(key + s" is required for $msg") + } + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala index 6596c5e2ad2e7..349cbd04f6027 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala @@ -118,6 +118,17 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { assert(e.getMessage.contains("hostPath.volumeName.options.path")) } + test("SPARK-33063: Fails on missing option key in persistentVolumeClaim") { + val sparkConf = new SparkConf(false) + sparkConf.set("test.persistentVolumeClaim.volumeName.mount.path", "/path") + sparkConf.set("test.persistentVolumeClaim.volumeName.mount.readOnly", "true") + + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("persistentVolumeClaim.volumeName.options.claimName")) + } + test("Parses read-only nfs volumes correctly") { val sparkConf = new SparkConf(false) sparkConf.set("test.nfs.volumeName.mount.path", "/path") From 14aeab3b279b1c23cddb86b97afc048c195b9b75 Mon Sep 17 00:00:00 2001 From: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com> Date: Mon, 5 Oct 2020 09:30:27 -0700 Subject: [PATCH 39/59] [SPARK-33038][SQL] Combine AQE initial and current plan string when two plans are the same ### What changes were proposed in this pull request? This PR combines the current plan and the initial plan in the AQE query plan string when the two plans are the same. It also removes the `== Current Plan ==` and `== Initial Plan ==` headers: Before ```scala AdaptiveSparkPlan isFinalPlan=false +- == Current Plan == SortMergeJoin [key#13], [a#23], Inner :- Sort [key#13 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(key#13, 5), true, [id=#94] ... +- == Initial Plan == SortMergeJoin [key#13], [a#23], Inner :- Sort [key#13 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(key#13, 5), true, [id=#94] ... ``` After ```scala AdaptiveSparkPlan isFinalPlan=false +- SortMergeJoin [key#13], [a#23], Inner :- Sort [key#13 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(key#13, 5), true, [id=#94] ... ``` For SQL `EXPLAIN` output: Before ```scala AdaptiveSparkPlan (8) +- == Current Plan == Sort (7) +- Exchange (6) ... +- == Initial Plan == Sort (7) +- Exchange (6) ... ``` After ```scala AdaptiveSparkPlan (8) +- Sort (7) +- Exchange (6) ... ``` ### Why are the changes needed? To simplify the AQE plan string by removing the redundant plan information. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Modified the existing unit test. Closes #29915 from allisonwang-db/aqe-explain. Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com> Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 50 ++++--- .../sql-tests/results/explain-aqe.sql.out | 123 ++---------------- .../adaptive/AdaptiveQueryExecSuite.scala | 4 +- 3 files changed, 47 insertions(+), 130 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 6c197fedd8c56..0e032569bb8a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -300,26 +300,40 @@ case class AdaptiveSparkPlanExec( maxFields, printNodeId, indent) - generateTreeStringWithHeader( - if (isFinalPlan) "Final Plan" else "Current Plan", - currentPhysicalPlan, - depth, - lastChildren, - append, - verbose, - maxFields, - printNodeId) - generateTreeStringWithHeader( - "Initial Plan", - initialPlan, - depth, - lastChildren, - append, - verbose, - maxFields, - printNodeId) + if (currentPhysicalPlan.fastEquals(initialPlan)) { + currentPhysicalPlan.generateTreeString( + depth + 1, + lastChildren :+ true, + append, + verbose, + prefix = "", + addSuffix = false, + maxFields, + printNodeId, + indent) + } else { + generateTreeStringWithHeader( + if (isFinalPlan) "Final Plan" else "Current Plan", + currentPhysicalPlan, + depth, + lastChildren, + append, + verbose, + maxFields, + printNodeId) + generateTreeStringWithHeader( + "Initial Plan", + initialPlan, + depth, + lastChildren, + append, + verbose, + maxFields, + printNodeId) + } } + private def generateTreeStringWithHeader( header: String, plan: SparkPlan, diff --git a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out index 3a850160b43e0..5435cde050fd1 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out @@ -54,16 +54,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (8) -+- == Current Plan == - Sort (7) - +- Exchange (6) - +- HashAggregate (5) - +- Exchange (4) - +- HashAggregate (3) - +- Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Sort (7) ++- Sort (7) +- Exchange (6) +- HashAggregate (5) +- Exchange (4) @@ -126,16 +117,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (8) -+- == Current Plan == - Project (7) - +- Filter (6) - +- HashAggregate (5) - +- Exchange (4) - +- HashAggregate (3) - +- Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Project (7) ++- Project (7) +- Filter (6) +- HashAggregate (5) +- Exchange (4) @@ -196,17 +178,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (9) -+- == Current Plan == - HashAggregate (8) - +- Exchange (7) - +- HashAggregate (6) - +- Union (5) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- Filter (4) - +- Scan parquet default.explain_temp1 (3) -+- == Initial Plan == - HashAggregate (8) ++- HashAggregate (8) +- Exchange (7) +- HashAggregate (6) +- Union (5) @@ -274,15 +246,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (6) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (5) - +- Filter (4) - +- Scan parquet default.explain_temp2 (3) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (6) ++- BroadcastHashJoin Inner BuildRight (6) :- Filter (2) : +- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (5) @@ -337,14 +301,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (6) -+- == Current Plan == - BroadcastHashJoin LeftOuter BuildRight (5) - :- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (4) - +- Filter (3) - +- Scan parquet default.explain_temp2 (2) -+- == Initial Plan == - BroadcastHashJoin LeftOuter BuildRight (5) ++- BroadcastHashJoin LeftOuter BuildRight (5) :- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (4) +- Filter (3) @@ -398,11 +355,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Filter (2) ++- Filter (2) +- Scan parquet default.explain_temp1 (1) @@ -438,11 +391,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Filter (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Filter (2) ++- Filter (2) +- Scan parquet default.explain_temp1 (1) @@ -470,11 +419,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (3) -+- == Current Plan == - Project (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - Project (2) ++- Project (2) +- Scan parquet default.explain_temp1 (1) @@ -506,15 +451,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (6) - :- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (5) - +- Filter (4) - +- Scan parquet default.explain_temp1 (3) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (6) ++- BroadcastHashJoin Inner BuildRight (6) :- Filter (2) : +- Scan parquet default.explain_temp1 (1) +- BroadcastExchange (5) @@ -572,21 +509,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (13) -+- == Current Plan == - BroadcastHashJoin Inner BuildRight (12) - :- HashAggregate (5) - : +- Exchange (4) - : +- HashAggregate (3) - : +- Filter (2) - : +- Scan parquet default.explain_temp1 (1) - +- BroadcastExchange (11) - +- HashAggregate (10) - +- Exchange (9) - +- HashAggregate (8) - +- Filter (7) - +- Scan parquet default.explain_temp1 (6) -+- == Initial Plan == - BroadcastHashJoin Inner BuildRight (12) ++- BroadcastHashJoin Inner BuildRight (12) :- HashAggregate (5) : +- Exchange (4) : +- HashAggregate (3) @@ -710,13 +633,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (5) -+- == Current Plan == - HashAggregate (4) - +- Exchange (3) - +- HashAggregate (2) - +- Scan parquet default.explain_temp1 (1) -+- == Initial Plan == - HashAggregate (4) ++- HashAggregate (4) +- Exchange (3) +- HashAggregate (2) +- Scan parquet default.explain_temp1 (1) @@ -761,13 +678,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (5) -+- == Current Plan == - ObjectHashAggregate (4) - +- Exchange (3) - +- ObjectHashAggregate (2) - +- Scan parquet default.explain_temp4 (1) -+- == Initial Plan == - ObjectHashAggregate (4) ++- ObjectHashAggregate (4) +- Exchange (3) +- ObjectHashAggregate (2) +- Scan parquet default.explain_temp4 (1) @@ -812,15 +723,7 @@ struct -- !query output == Physical Plan == AdaptiveSparkPlan (7) -+- == Current Plan == - SortAggregate (6) - +- Sort (5) - +- Exchange (4) - +- SortAggregate (3) - +- Sort (2) - +- Scan parquet default.explain_temp4 (1) -+- == Initial Plan == - SortAggregate (6) ++- SortAggregate (6) +- Sort (5) +- Exchange (4) +- SortAggregate (3) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 8799dbb14ef34..0dfb1d2fd9eda 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -842,8 +842,8 @@ class AdaptiveQueryExecSuite withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val df = sql("SELECT * FROM testData join testData2 ON key = a where value = '1'") val planBefore = df.queryExecution.executedPlan - assert(planBefore.toString.contains("== Current Plan ==")) - assert(planBefore.toString.contains("== Initial Plan ==")) + assert(!planBefore.toString.contains("== Current Plan ==")) + assert(!planBefore.toString.contains("== Initial Plan ==")) df.collect() val planAfter = df.queryExecution.executedPlan assert(planAfter.toString.contains("== Final Plan ==")) From 008a2ad1f836ff04fafd51a9c94c355ef35f1692 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 5 Oct 2020 15:29:56 -0700 Subject: [PATCH 40/59] [SPARK-20202][BUILD][SQL] Remove references to org.spark-project.hive (Hive 1.2.1) ### What changes were proposed in this pull request? As of today, - SPARK-30034 Apache Spark 3.0.0 switched its default Hive execution engine from Hive 1.2 to Hive 2.3. This removes the direct dependency to the forked Hive 1.2.1 in maven repository. - SPARK-32981 Apache Spark 3.1.0(`master` branch) removed Hive 1.2 related artifacts from Apache Spark binary distributions. This PR(SPARK-20202) aims to remove the following usage of unofficial Apache Hive fork completely from Apache Spark master for Apache Spark 3.1.0. ``` org.spark-project.hive 1.2.1.spark2 ``` For the forked Hive 1.2.1.spark2 users, Apache Spark 2.4(LTS) and 3.0 (~ 2021.12) will provide it. ### Why are the changes needed? - First, Apache Spark community should not use the unofficial forked release of another Apache project. - Second, Apache Hive 1.2.1 was released at 2015-06-26 and the forked Hive `1.2.1.spark2` exposed many unfixable bugs in Apache because the forked `1.2.1.spark2` is not maintained at all. Apache Hive 2.3.0 was released at 2017-07-19 and it has been used with less number of bugs compared with `1.2.1.spark2`. Many bugs still exist in `hive-1.2` profile and new Apache Spark unit tests are added with `HiveUtils.isHive23` condition so far. ### Does this PR introduce _any_ user-facing change? No. This is a dev-only change. PRBuilder will not accept `[test-hive1.2]` on master and `branch-3.1`. ### How was this patch tested? 1. SBT/Hadoop 3.2/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129366) 2. SBT/Hadoop 2.7/Hive 2.3 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129382) 3. SBT/Hadoop 3.2/Hive 1.2 (This has not been supported already due to Hive 1.2 doesn't work with Hadoop 3.2.) 4. SBT/Hadoop 2.7/Hive 1.2 (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129383, This is rejected) Closes #29936 from dongjoon-hyun/SPARK-REMOVE-HIVE1. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/run-tests.py | 1 - dev/test-dependencies.sh | 6 +- docs/sql-migration-guide.md | 2 + pom.xml | 25 - sql/core/pom.xml | 3 - .../datasources/orc/OrcColumnVector.java | 0 .../datasources/orc/OrcFilters.scala | 0 .../datasources/orc/OrcShimUtils.scala | 0 .../datasources/orc/OrcFilterSuite.scala | 0 .../datasources/orc/OrcColumnVector.java | 208 - .../datasources/orc/DaysWritable.scala | 79 - .../datasources/orc/OrcFilters.scala | 275 - .../datasources/orc/OrcShimUtils.scala | 66 - .../datasources/orc/OrcFilterSuite.scala | 676 - .../{v2.3 => }/if/TCLIService.thrift | 0 sql/hive-thriftserver/pom.xml | 4 +- .../service/rpc/thrift/TArrayTypeEntry.java | 0 .../service/rpc/thrift/TBinaryColumn.java | 0 .../hive/service/rpc/thrift/TBoolColumn.java | 0 .../hive/service/rpc/thrift/TBoolValue.java | 0 .../hive/service/rpc/thrift/TByteColumn.java | 0 .../hive/service/rpc/thrift/TByteValue.java | 0 .../hive/service/rpc/thrift/TCLIService.java | 0 .../rpc/thrift/TCLIServiceConstants.java | 0 .../rpc/thrift/TCancelDelegationTokenReq.java | 0 .../thrift/TCancelDelegationTokenResp.java | 0 .../rpc/thrift/TCancelOperationReq.java | 0 .../rpc/thrift/TCancelOperationResp.java | 0 .../rpc/thrift/TCloseOperationReq.java | 0 .../rpc/thrift/TCloseOperationResp.java | 0 .../service/rpc/thrift/TCloseSessionReq.java | 0 .../service/rpc/thrift/TCloseSessionResp.java | 0 .../hive/service/rpc/thrift/TColumn.java | 0 .../hive/service/rpc/thrift/TColumnDesc.java | 0 .../hive/service/rpc/thrift/TColumnValue.java | 0 .../service/rpc/thrift/TDoubleColumn.java | 0 .../hive/service/rpc/thrift/TDoubleValue.java | 0 .../rpc/thrift/TExecuteStatementReq.java | 0 .../rpc/thrift/TExecuteStatementResp.java | 0 .../service/rpc/thrift/TFetchOrientation.java | 0 .../service/rpc/thrift/TFetchResultsReq.java | 0 .../service/rpc/thrift/TFetchResultsResp.java | 0 .../service/rpc/thrift/TGetCatalogsReq.java | 0 .../service/rpc/thrift/TGetCatalogsResp.java | 0 .../service/rpc/thrift/TGetColumnsReq.java | 0 .../service/rpc/thrift/TGetColumnsResp.java | 0 .../rpc/thrift/TGetCrossReferenceReq.java | 0 .../rpc/thrift/TGetCrossReferenceResp.java | 0 .../rpc/thrift/TGetDelegationTokenReq.java | 0 .../rpc/thrift/TGetDelegationTokenResp.java | 0 .../service/rpc/thrift/TGetFunctionsReq.java | 0 .../service/rpc/thrift/TGetFunctionsResp.java | 0 .../hive/service/rpc/thrift/TGetInfoReq.java | 0 .../hive/service/rpc/thrift/TGetInfoResp.java | 0 .../hive/service/rpc/thrift/TGetInfoType.java | 0 .../service/rpc/thrift/TGetInfoValue.java | 0 .../rpc/thrift/TGetOperationStatusReq.java | 0 .../rpc/thrift/TGetOperationStatusResp.java | 0 .../rpc/thrift/TGetPrimaryKeysReq.java | 0 .../rpc/thrift/TGetPrimaryKeysResp.java | 0 .../rpc/thrift/TGetResultSetMetadataReq.java | 0 .../rpc/thrift/TGetResultSetMetadataResp.java | 0 .../service/rpc/thrift/TGetSchemasReq.java | 0 .../service/rpc/thrift/TGetSchemasResp.java | 0 .../service/rpc/thrift/TGetTableTypesReq.java | 0 .../rpc/thrift/TGetTableTypesResp.java | 0 .../service/rpc/thrift/TGetTablesReq.java | 0 .../service/rpc/thrift/TGetTablesResp.java | 0 .../service/rpc/thrift/TGetTypeInfoReq.java | 0 .../service/rpc/thrift/TGetTypeInfoResp.java | 0 .../service/rpc/thrift/THandleIdentifier.java | 0 .../hive/service/rpc/thrift/TI16Column.java | 0 .../hive/service/rpc/thrift/TI16Value.java | 0 .../hive/service/rpc/thrift/TI32Column.java | 0 .../hive/service/rpc/thrift/TI32Value.java | 0 .../hive/service/rpc/thrift/TI64Column.java | 0 .../hive/service/rpc/thrift/TI64Value.java | 0 .../rpc/thrift/TJobExecutionStatus.java | 0 .../service/rpc/thrift/TMapTypeEntry.java | 0 .../service/rpc/thrift/TOpenSessionReq.java | 0 .../service/rpc/thrift/TOpenSessionResp.java | 0 .../service/rpc/thrift/TOperationHandle.java | 0 .../service/rpc/thrift/TOperationState.java | 0 .../service/rpc/thrift/TOperationType.java | 0 .../rpc/thrift/TPrimitiveTypeEntry.java | 0 .../rpc/thrift/TProgressUpdateResp.java | 0 .../service/rpc/thrift/TProtocolVersion.java | 0 .../rpc/thrift/TRenewDelegationTokenReq.java | 0 .../rpc/thrift/TRenewDelegationTokenResp.java | 0 .../apache/hive/service/rpc/thrift/TRow.java | 0 .../hive/service/rpc/thrift/TRowSet.java | 0 .../service/rpc/thrift/TSessionHandle.java | 0 .../hive/service/rpc/thrift/TStatus.java | 0 .../hive/service/rpc/thrift/TStatusCode.java | 0 .../service/rpc/thrift/TStringColumn.java | 0 .../hive/service/rpc/thrift/TStringValue.java | 0 .../service/rpc/thrift/TStructTypeEntry.java | 0 .../hive/service/rpc/thrift/TTableSchema.java | 0 .../hive/service/rpc/thrift/TTypeDesc.java | 0 .../hive/service/rpc/thrift/TTypeEntry.java | 0 .../hive/service/rpc/thrift/TTypeId.java | 0 .../rpc/thrift/TTypeQualifierValue.java | 0 .../service/rpc/thrift/TTypeQualifiers.java | 0 .../service/rpc/thrift/TUnionTypeEntry.java | 0 .../rpc/thrift/TUserDefinedTypeEntry.java | 0 .../apache/hive/service/AbstractService.java | 0 .../apache/hive/service/CompositeService.java | 0 .../org/apache/hive/service/CookieSigner.java | 0 .../hive/service/ServiceOperations.java | 0 .../org/apache/hive/service/ServiceUtils.java | 0 .../hive/service/auth/HiveAuthFactory.java | 0 .../hive/service/auth/HttpAuthUtils.java | 0 .../hive/service/auth/KerberosSaslHelper.java | 0 .../hive/service/auth/PlainSaslHelper.java | 0 .../service/auth/TSetIpAddressProcessor.java | 0 .../apache/hive/service/cli/CLIService.java | 0 .../hive/service/cli/ColumnBasedSet.java | 0 .../hive/service/cli/ColumnDescriptor.java | 0 .../apache/hive/service/cli/ColumnValue.java | 0 .../hive/service/cli/FetchOrientation.java | 0 .../apache/hive/service/cli/GetInfoType.java | 0 .../apache/hive/service/cli/GetInfoValue.java | 0 .../org/apache/hive/service/cli/Handle.java | 0 .../hive/service/cli/HandleIdentifier.java | 0 .../hive/service/cli/HiveSQLException.java | 0 .../apache/hive/service/cli/ICLIService.java | 0 .../hive/service/cli/OperationHandle.java | 0 .../hive/service/cli/OperationState.java | 0 .../hive/service/cli/OperationType.java | 0 .../apache/hive/service/cli/RowBasedSet.java | 0 .../org/apache/hive/service/cli/RowSet.java | 0 .../hive/service/cli/RowSetFactory.java | 0 .../hive/service/cli/SessionHandle.java | 0 .../apache/hive/service/cli/TableSchema.java | 0 .../hive/service/cli/TypeDescriptor.java | 0 .../hive/service/cli/TypeQualifiers.java | 0 .../operation/ClassicTableTypeMapping.java | 0 .../operation/ExecuteStatementOperation.java | 0 .../cli/operation/GetCatalogsOperation.java | 0 .../cli/operation/GetColumnsOperation.java | 0 .../operation/GetCrossReferenceOperation.java | 0 .../cli/operation/GetFunctionsOperation.java | 0 .../operation/GetPrimaryKeysOperation.java | 0 .../cli/operation/GetSchemasOperation.java | 0 .../cli/operation/GetTableTypesOperation.java | 0 .../cli/operation/GetTablesOperation.java | 0 .../cli/operation/GetTypeInfoOperation.java | 0 .../cli/operation/HiveCommandOperation.java | 0 .../cli/operation/HiveTableTypeMapping.java | 0 .../cli/operation/MetadataOperation.java | 0 .../hive/service/cli/operation/Operation.java | 0 .../cli/operation/OperationManager.java | 0 .../service/cli/operation/SQLOperation.java | 0 .../cli/operation/TableTypeMapping.java | 0 .../hive/service/cli/session/HiveSession.java | 0 .../service/cli/session/HiveSessionBase.java | 0 .../cli/session/HiveSessionHookContext.java | 0 .../session/HiveSessionHookContextImpl.java | 0 .../service/cli/session/HiveSessionImpl.java | 0 .../cli/session/HiveSessionImplwithUGI.java | 0 .../service/cli/session/SessionManager.java | 0 .../cli/thrift/ThriftBinaryCLIService.java | 0 .../service/cli/thrift/ThriftCLIService.java | 0 .../cli/thrift/ThriftCLIServiceClient.java | 0 .../cli/thrift/ThriftHttpCLIService.java | 0 .../service/cli/thrift/ThriftHttpServlet.java | 0 .../hive/service/server/HiveServer2.java | 0 .../server/ThreadWithGarbageCleanup.java | 0 .../thriftserver/ThriftserverShimUtils.scala | 0 .../v1.2/if/TCLIService.thrift | 1173 -- .../service/cli/thrift/TArrayTypeEntry.java | 383 - .../service/cli/thrift/TBinaryColumn.java | 550 - .../hive/service/cli/thrift/TBoolColumn.java | 548 - .../hive/service/cli/thrift/TBoolValue.java | 386 - .../hive/service/cli/thrift/TByteColumn.java | 548 - .../hive/service/cli/thrift/TByteValue.java | 386 - .../hive/service/cli/thrift/TCLIService.java | 15414 ---------------- .../cli/thrift/TCLIServiceConstants.java | 103 - .../cli/thrift/TCancelDelegationTokenReq.java | 491 - .../thrift/TCancelDelegationTokenResp.java | 390 - .../cli/thrift/TCancelOperationReq.java | 390 - .../cli/thrift/TCancelOperationResp.java | 390 - .../cli/thrift/TCloseOperationReq.java | 390 - .../cli/thrift/TCloseOperationResp.java | 390 - .../service/cli/thrift/TCloseSessionReq.java | 390 - .../service/cli/thrift/TCloseSessionResp.java | 390 - .../hive/service/cli/thrift/TColumn.java | 732 - .../hive/service/cli/thrift/TColumnDesc.java | 700 - .../hive/service/cli/thrift/TColumnValue.java | 671 - .../service/cli/thrift/TDoubleColumn.java | 548 - .../hive/service/cli/thrift/TDoubleValue.java | 386 - .../cli/thrift/TExecuteStatementReq.java | 769 - .../cli/thrift/TExecuteStatementResp.java | 505 - .../service/cli/thrift/TFetchOrientation.java | 57 - .../service/cli/thrift/TFetchResultsReq.java | 710 - .../service/cli/thrift/TFetchResultsResp.java | 608 - .../service/cli/thrift/TGetCatalogsReq.java | 390 - .../service/cli/thrift/TGetCatalogsResp.java | 505 - .../service/cli/thrift/TGetColumnsReq.java | 818 - .../service/cli/thrift/TGetColumnsResp.java | 505 - .../cli/thrift/TGetDelegationTokenReq.java | 592 - .../cli/thrift/TGetDelegationTokenResp.java | 500 - .../service/cli/thrift/TGetFunctionsReq.java | 707 - .../service/cli/thrift/TGetFunctionsResp.java | 505 - .../hive/service/cli/thrift/TGetInfoReq.java | 503 - .../hive/service/cli/thrift/TGetInfoResp.java | 493 - .../hive/service/cli/thrift/TGetInfoType.java | 180 - .../service/cli/thrift/TGetInfoValue.java | 593 - .../cli/thrift/TGetOperationStatusReq.java | 390 - .../cli/thrift/TGetOperationStatusResp.java | 827 - .../cli/thrift/TGetResultSetMetadataReq.java | 390 - .../cli/thrift/TGetResultSetMetadataResp.java | 505 - .../service/cli/thrift/TGetSchemasReq.java | 606 - .../service/cli/thrift/TGetSchemasResp.java | 505 - .../service/cli/thrift/TGetTableTypesReq.java | 390 - .../cli/thrift/TGetTableTypesResp.java | 505 - .../service/cli/thrift/TGetTablesReq.java | 870 - .../service/cli/thrift/TGetTablesResp.java | 505 - .../service/cli/thrift/TGetTypeInfoReq.java | 390 - .../service/cli/thrift/TGetTypeInfoResp.java | 505 - .../service/cli/thrift/THandleIdentifier.java | 506 - .../hive/service/cli/thrift/TI16Column.java | 548 - .../hive/service/cli/thrift/TI16Value.java | 386 - .../hive/service/cli/thrift/TI32Column.java | 548 - .../hive/service/cli/thrift/TI32Value.java | 386 - .../hive/service/cli/thrift/TI64Column.java | 548 - .../hive/service/cli/thrift/TI64Value.java | 386 - .../service/cli/thrift/TMapTypeEntry.java | 478 - .../service/cli/thrift/TOpenSessionReq.java | 785 - .../service/cli/thrift/TOpenSessionResp.java | 790 - .../service/cli/thrift/TOperationHandle.java | 705 - .../service/cli/thrift/TOperationState.java | 63 - .../service/cli/thrift/TOperationType.java | 66 - .../cli/thrift/TPrimitiveTypeEntry.java | 512 - .../service/cli/thrift/TProtocolVersion.java | 63 - .../cli/thrift/TRenewDelegationTokenReq.java | 491 - .../cli/thrift/TRenewDelegationTokenResp.java | 390 - .../apache/hive/service/cli/thrift/TRow.java | 439 - .../hive/service/cli/thrift/TRowSet.java | 702 - .../service/cli/thrift/TSessionHandle.java | 390 - .../hive/service/cli/thrift/TStatus.java | 874 - .../hive/service/cli/thrift/TStatusCode.java | 54 - .../service/cli/thrift/TStringColumn.java | 548 - .../hive/service/cli/thrift/TStringValue.java | 389 - .../service/cli/thrift/TStructTypeEntry.java | 448 - .../hive/service/cli/thrift/TTableSchema.java | 439 - .../hive/service/cli/thrift/TTypeDesc.java | 439 - .../hive/service/cli/thrift/TTypeEntry.java | 610 - .../hive/service/cli/thrift/TTypeId.java | 105 - .../cli/thrift/TTypeQualifierValue.java | 361 - .../service/cli/thrift/TTypeQualifiers.java | 450 - .../service/cli/thrift/TUnionTypeEntry.java | 448 - .../cli/thrift/TUserDefinedTypeEntry.java | 385 - .../apache/hive/service/AbstractService.java | 184 - .../apache/hive/service/CompositeService.java | 133 - .../org/apache/hive/service/CookieSigner.java | 108 - .../hive/service/ServiceOperations.java | 141 - .../org/apache/hive/service/ServiceUtils.java | 44 - .../hive/service/auth/HiveAuthFactory.java | 419 - .../hive/service/auth/HttpAuthUtils.java | 189 - .../hive/service/auth/KerberosSaslHelper.java | 111 - .../hive/service/auth/PlainSaslHelper.java | 154 - .../service/auth/TSetIpAddressProcessor.java | 114 - .../apache/hive/service/cli/CLIService.java | 507 - .../org/apache/hive/service/cli/Column.java | 423 - .../hive/service/cli/ColumnBasedSet.java | 149 - .../hive/service/cli/ColumnDescriptor.java | 99 - .../apache/hive/service/cli/ColumnValue.java | 288 - .../service/cli/EmbeddedCLIServiceClient.java | 208 - .../hive/service/cli/FetchOrientation.java | 54 - .../apache/hive/service/cli/GetInfoType.java | 96 - .../apache/hive/service/cli/GetInfoValue.java | 82 - .../org/apache/hive/service/cli/Handle.java | 78 - .../hive/service/cli/HandleIdentifier.java | 113 - .../hive/service/cli/HiveSQLException.java | 249 - .../apache/hive/service/cli/ICLIService.java | 105 - .../hive/service/cli/OperationHandle.java | 102 - .../hive/service/cli/OperationState.java | 108 - .../hive/service/cli/OperationType.java | 58 - .../hive/service/cli/PatternOrIdentifier.java | 47 - .../apache/hive/service/cli/RowBasedSet.java | 140 - .../org/apache/hive/service/cli/RowSet.java | 38 - .../hive/service/cli/RowSetFactory.java | 41 - .../hive/service/cli/SessionHandle.java | 67 - .../apache/hive/service/cli/TableSchema.java | 102 - .../org/apache/hive/service/cli/Type.java | 349 - .../hive/service/cli/TypeDescriptor.java | 159 - .../hive/service/cli/TypeQualifiers.java | 133 - .../operation/ClassicTableTypeMapping.java | 86 - .../operation/ExecuteStatementOperation.java | 83 - .../cli/operation/GetCatalogsOperation.java | 81 - .../cli/operation/GetColumnsOperation.java | 234 - .../cli/operation/GetFunctionsOperation.java | 147 - .../cli/operation/GetSchemasOperation.java | 96 - .../cli/operation/GetTableTypesOperation.java | 93 - .../cli/operation/GetTablesOperation.java | 135 - .../cli/operation/GetTypeInfoOperation.java | 142 - .../cli/operation/HiveCommandOperation.java | 215 - .../cli/operation/HiveTableTypeMapping.java | 51 - .../cli/operation/MetadataOperation.java | 134 - .../hive/service/cli/operation/Operation.java | 328 - .../cli/operation/OperationManager.java | 284 - .../service/cli/operation/SQLOperation.java | 456 - .../cli/operation/TableTypeMapping.java | 44 - .../hive/service/cli/session/HiveSession.java | 156 - .../service/cli/session/HiveSessionBase.java | 90 - .../service/cli/session/HiveSessionImpl.java | 842 - .../cli/session/HiveSessionImplwithUGI.java | 182 - .../service/cli/session/SessionManager.java | 377 - .../cli/thrift/ThriftBinaryCLIService.java | 121 - .../service/cli/thrift/ThriftCLIService.java | 693 - .../cli/thrift/ThriftCLIServiceClient.java | 440 - .../cli/thrift/ThriftHttpCLIService.java | 194 - .../service/cli/thrift/ThriftHttpServlet.java | 545 - .../hive/service/server/HiveServer2.java | 277 - .../server/ThreadWithGarbageCleanup.java | 77 - .../thriftserver/ThriftserverShimUtils.scala | 77 - ...IntoHiveTableBenchmark-hive1.2-results.txt | 11 - .../sql/hive/client/HiveClientImpl.scala | 3 +- .../InsertIntoHiveTableBenchmark.scala | 7 +- 320 files changed, 7 insertions(+), 69240 deletions(-) rename sql/core/{v2.3 => }/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java (100%) rename sql/core/{v2.3 => }/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala (100%) rename sql/core/{v2.3 => }/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala (100%) rename sql/core/{v2.3 => }/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala (100%) delete mode 100644 sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java delete mode 100644 sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala delete mode 100644 sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala delete mode 100644 sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala delete mode 100644 sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename sql/hive-thriftserver/{v2.3 => }/if/TCLIService.thrift (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/AbstractService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/CompositeService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/CookieSigner.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/ServiceOperations.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/ServiceUtils.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/CLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/ColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/FetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/GetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/GetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/Handle.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/HiveSQLException.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/ICLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/OperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/OperationState.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/OperationType.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/RowBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/RowSet.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/SessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/TableSchema.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/Operation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSession.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/session/SessionManager.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/server/HiveServer2.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (100%) rename sql/hive-thriftserver/{v2.3 => }/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala (100%) delete mode 100644 sql/hive-thriftserver/v1.2/if/TCLIService.thrift delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java delete mode 100644 sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala delete mode 100644 sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt diff --git a/dev/run-tests.py b/dev/run-tests.py index 3e118dcbc160d..48191e9bb024d 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -325,7 +325,6 @@ def get_hive_profiles(hive_version): """ sbt_maven_hive_profiles = { - "hive1.2": ["-Phive-1.2"], "hive2.3": ["-Phive-2.3"], } diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 129b073d75254..e9e9227d239e1 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -32,7 +32,6 @@ export LC_ALL=C HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" MVN="build/mvn" HADOOP_HIVE_PROFILES=( - hadoop-2.7-hive-1.2 hadoop-2.7-hive-2.3 hadoop-3.2-hive-2.3 ) @@ -71,12 +70,9 @@ for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do if [[ $HADOOP_HIVE_PROFILE == **hadoop-3.2-hive-2.3** ]]; then HADOOP_PROFILE=hadoop-3.2 HIVE_PROFILE=hive-2.3 - elif [[ $HADOOP_HIVE_PROFILE == **hadoop-2.7-hive-2.3** ]]; then - HADOOP_PROFILE=hadoop-2.7 - HIVE_PROFILE=hive-2.3 else HADOOP_PROFILE=hadoop-2.7 - HIVE_PROFILE=hive-1.2 + HIVE_PROFILE=hive-2.3 fi echo "Performing Maven install for $HADOOP_HIVE_PROFILE" $MVN $HADOOP_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index de60aed7483c7..feff2c7e9f543 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -42,6 +42,8 @@ license: | - In Spark 3.1, incomplete interval literals, e.g. `INTERVAL '1'`, `INTERVAL '1 DAY 2'` will fail with IllegalArgumentException. In Spark 3.0, they result `NULL`s. + - In Spark 3.1, we remove the built-in Hive 1.2. You need to migrate your custom SerDes to Hive 2.3. See [HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) for more details. + ## Upgrading from Spark SQL 3.0 to 3.0.1 - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON option `inferTimestamp` to `true` to enable such type inference. diff --git a/pom.xml b/pom.xml index 5d6b0511ce458..b13d5ab81856c 100644 --- a/pom.xml +++ b/pom.xml @@ -2970,13 +2970,9 @@ ${basedir}/src/main/java ${basedir}/src/main/scala - ${basedir}/v${hive.version.short}/src/main/java - ${basedir}/v${hive.version.short}/src/main/scala ${basedir}/src/test/java - ${basedir}/v${hive.version.short}/src/test/java - ${basedir}/v${hive.version.short}/src/test/scala dev/checkstyle.xml ${basedir}/target/checkstyle-output.xml @@ -3148,27 +3144,6 @@ - - hive-1.2 - - org.spark-project.hive - - - 1.2.1.spark2 - - 1.2 - ${hive.deps.scope} - 2.6.0 - provided - provided - provided - provided - provided - nohive - 3.2.10 - - - hive-2.3 diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c2ed4c079d3cf..0f5d3fd55c15d 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -221,8 +221,6 @@ - v${hive.version.short}/src/main/scala - v${hive.version.short}/src/main/java src/main/scala-${scala.binary.version} @@ -235,7 +233,6 @@ - v${hive.version.short}/src/test/scala src/test/gen-java diff --git a/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 100% rename from sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala diff --git a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java deleted file mode 100644 index 6601bcb9018f4..0000000000000 --- a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.spark.sql.execution.datasources.orc; - -import java.math.BigDecimal; - -import org.apache.orc.storage.ql.exec.vector.*; - -import org.apache.spark.sql.catalyst.util.DateTimeUtils; -import org.apache.spark.sql.catalyst.util.RebaseDateTime; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DateType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.TimestampType; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -/** - * A column vector class wrapping Hive's ColumnVector. Because Spark ColumnarBatch only accepts - * Spark's vectorized.ColumnVector, this column vector is used to adapt Hive ColumnVector with - * Spark ColumnarVector. - */ -public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVector { - private ColumnVector baseData; - private LongColumnVector longData; - private DoubleColumnVector doubleData; - private BytesColumnVector bytesData; - private DecimalColumnVector decimalData; - private TimestampColumnVector timestampData; - private final boolean isTimestamp; - private final boolean isDate; - - private int batchSize; - - OrcColumnVector(DataType type, ColumnVector vector) { - super(type); - - if (type instanceof TimestampType) { - isTimestamp = true; - } else { - isTimestamp = false; - } - - if (type instanceof DateType) { - isDate = true; - } else { - isDate = false; - } - - baseData = vector; - if (vector instanceof LongColumnVector) { - longData = (LongColumnVector) vector; - } else if (vector instanceof DoubleColumnVector) { - doubleData = (DoubleColumnVector) vector; - } else if (vector instanceof BytesColumnVector) { - bytesData = (BytesColumnVector) vector; - } else if (vector instanceof DecimalColumnVector) { - decimalData = (DecimalColumnVector) vector; - } else if (vector instanceof TimestampColumnVector) { - timestampData = (TimestampColumnVector) vector; - } else { - throw new UnsupportedOperationException(); - } - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - @Override - public void close() { - - } - - @Override - public boolean hasNull() { - return !baseData.noNulls; - } - - @Override - public int numNulls() { - if (baseData.isRepeating) { - if (baseData.isNull[0]) { - return batchSize; - } else { - return 0; - } - } else if (baseData.noNulls) { - return 0; - } else { - int count = 0; - for (int i = 0; i < batchSize; i++) { - if (baseData.isNull[i]) count++; - } - return count; - } - } - - /* A helper method to get the row index in a column. */ - private int getRowIndex(int rowId) { - return baseData.isRepeating ? 0 : rowId; - } - - @Override - public boolean isNullAt(int rowId) { - return baseData.isNull[getRowIndex(rowId)]; - } - - @Override - public boolean getBoolean(int rowId) { - return longData.vector[getRowIndex(rowId)] == 1; - } - - @Override - public byte getByte(int rowId) { - return (byte) longData.vector[getRowIndex(rowId)]; - } - - @Override - public short getShort(int rowId) { - return (short) longData.vector[getRowIndex(rowId)]; - } - - @Override - public int getInt(int rowId) { - int value = (int) longData.vector[getRowIndex(rowId)]; - if (isDate) { - return RebaseDateTime.rebaseJulianToGregorianDays(value); - } else { - return value; - } - } - - @Override - public long getLong(int rowId) { - int index = getRowIndex(rowId); - if (isTimestamp) { - return DateTimeUtils.fromJavaTimestamp(timestampData.asScratchTimestamp(index)); - } else { - return longData.vector[index]; - } - } - - @Override - public float getFloat(int rowId) { - return (float) doubleData.vector[getRowIndex(rowId)]; - } - - @Override - public double getDouble(int rowId) { - return doubleData.vector[getRowIndex(rowId)]; - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - if (isNullAt(rowId)) return null; - BigDecimal data = decimalData.vector[getRowIndex(rowId)].getHiveDecimal().bigDecimalValue(); - return Decimal.apply(data, precision, scale); - } - - @Override - public UTF8String getUTF8String(int rowId) { - if (isNullAt(rowId)) return null; - int index = getRowIndex(rowId); - BytesColumnVector col = bytesData; - return UTF8String.fromBytes(col.vector[index], col.start[index], col.length[index]); - } - - @Override - public byte[] getBinary(int rowId) { - if (isNullAt(rowId)) return null; - int index = getRowIndex(rowId); - byte[] binary = new byte[bytesData.length[index]]; - System.arraycopy(bytesData.vector[index], bytesData.start[index], binary, 0, binary.length); - return binary; - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public org.apache.spark.sql.vectorized.ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala deleted file mode 100644 index 1dccf0ca1faef..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.spark.sql.execution.datasources.orc - -import java.io.{DataInput, DataOutput, IOException} -import java.sql.Date - -import org.apache.hadoop.io.WritableUtils -import org.apache.orc.storage.serde2.io.DateWritable - -import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} - -/** - * The class accepts/returns days in Gregorian calendar and rebase them - * via conversion to local date in Julian calendar for dates before 1582-10-15 - * in read/write for backward compatibility with Spark 2.4 and earlier versions. - * - * This is a clone of `org.apache.spark.sql.execution.datasources.DaysWritable`. - * The class is cloned because Hive ORC v1.2 uses different `DateWritable`: - * - v1.2: `org.apache.orc.storage.serde2.io.DateWritable` - * - v2.3 and `HiveInspectors`: `org.apache.hadoop.hive.serde2.io.DateWritable` - * - * @param gregorianDays The number of days since the epoch 1970-01-01 in - * Gregorian calendar. - * @param julianDays The number of days since the epoch 1970-01-01 in - * Julian calendar. - */ -class DaysWritable( - var gregorianDays: Int, - var julianDays: Int) - extends DateWritable { - - def this() = this(0, 0) - def this(gregorianDays: Int) = - this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) - def this(dateWritable: DateWritable) = { - this( - gregorianDays = dateWritable match { - case daysWritable: DaysWritable => daysWritable.gregorianDays - case dateWritable: DateWritable => - rebaseJulianToGregorianDays(dateWritable.getDays) - }, - julianDays = dateWritable.getDays) - } - - override def getDays: Int = julianDays - override def get(): Date = new Date(DateWritable.daysToMillis(julianDays)) - - override def set(d: Int): Unit = { - gregorianDays = d - julianDays = rebaseGregorianToJulianDays(d) - } - - @throws[IOException] - override def write(out: DataOutput): Unit = { - WritableUtils.writeVInt(out, julianDays) - } - - @throws[IOException] - override def readFields(in: DataInput): Unit = { - julianDays = WritableUtils.readVInt(in) - gregorianDays = rebaseJulianToGregorianDays(julianDays) - } -} diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala deleted file mode 100644 index 0e657bfe66238..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ /dev/null @@ -1,275 +0,0 @@ -/* - * 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.spark.sql.execution.datasources.orc - -import java.time.{Instant, LocalDate} - -import org.apache.orc.storage.common.`type`.HiveDecimal -import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder -import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder -import org.apache.orc.storage.serde2.io.HiveDecimalWritable - -import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types._ - -/** - * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. - * - * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and - * conversion passes through the Filter to make sure we only convert predicates that are known - * to be convertible. - * - * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't - * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite - * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using - * existing simpler ones. - * - * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and - * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, if we try to translate a filter - * before checking whether it can be converted or not, we may end up with a builder whose internal - * state is inconsistent in the case of an inconvertible filter. - * - * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then - * try to convert its children. Say we convert `left` child successfully, but find that `right` - * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent - * now. - * - * The workaround employed here is to trim the Spark filters before trying to convert them. This - * way, we can only do the actual conversion on the part of the Filter that is known to be - * convertible. - * - * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of - * builder methods mentioned above can only be found in test code, where all tested filters are - * known to be convertible. - */ -private[sql] object OrcFilters extends OrcFiltersBase { - - /** - * Create ORC filter as a SearchArgument instance. - */ - def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.getSearchableTypeMap(schema, SQLConf.get.caseSensitiveAnalysis) - // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) - conjunctionOptional.map { conjunction => - // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. - // The input predicate is fully convertible. There should not be any empty result in the - // following recursive method call `buildSearchArgument`. - buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() - } - } - - def convertibleFilters( - schema: StructType, - dataTypeMap: Map[String, OrcPrimitiveField], - filters: Seq[Filter]): Seq[Filter] = { - import org.apache.spark.sql.sources._ - - def convertibleFiltersHelper( - filter: Filter, - canPartialPushDown: Boolean): Option[Filter] = filter match { - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - case And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - case Or(left, right) => - for { - lhs <- convertibleFiltersHelper(left, canPartialPushDown) - rhs <- convertibleFiltersHelper(right, canPartialPushDown) - } yield Or(lhs, rhs) - case Not(pred) => - val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - childResultOptional.map(Not) - case other => - for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other - } - filters.flatMap { filter => - convertibleFiltersHelper(filter, true) - } - } - - /** - * Get PredicateLeafType which is corresponding to the given DataType. - */ - def getPredicateLeafType(dataType: DataType): PredicateLeaf.Type = dataType match { - case BooleanType => PredicateLeaf.Type.BOOLEAN - case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG - case FloatType | DoubleType => PredicateLeaf.Type.FLOAT - case StringType => PredicateLeaf.Type.STRING - case DateType => PredicateLeaf.Type.DATE - case TimestampType => PredicateLeaf.Type.TIMESTAMP - case _: DecimalType => PredicateLeaf.Type.DECIMAL - case _ => throw new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") - } - - /** - * Cast literal values for filters. - * - * We need to cast to long because ORC raises exceptions - * at 'checkLiteralType' of SearchArgumentImpl.java. - */ - private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { - case ByteType | ShortType | IntegerType | LongType => - value.asInstanceOf[Number].longValue - case FloatType | DoubleType => - value.asInstanceOf[Number].doubleValue() - case _: DecimalType => - new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) - case _: DateType if value.isInstanceOf[LocalDate] => - toJavaDate(localDateToDays(value.asInstanceOf[LocalDate])) - case _: TimestampType if value.isInstanceOf[Instant] => - toJavaTimestamp(instantToMicros(value.asInstanceOf[Instant])) - case _ => value - } - - /** - * Build a SearchArgument and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input predicates, which should be fully convertible to SearchArgument. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, OrcPrimitiveField], - expression: Filter, - builder: Builder): Builder = { - import org.apache.spark.sql.sources._ - - expression match { - case And(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Or(left, right) => - val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) - val rhs = buildSearchArgument(dataTypeMap, right, lhs) - rhs.end() - - case Not(child) => - buildSearchArgument(dataTypeMap, child, builder.startNot()).end() - - case other => - buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { - throw new SparkException( - "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") - } - } - } - - /** - * Build a SearchArgument for a leaf predicate and return the builder so far. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @return the builder so far. - */ - private def buildLeafSearchArgument( - dataTypeMap: Map[String, OrcPrimitiveField], - expression: Filter, - builder: Builder): Option[Builder] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute).fieldType) - - import org.apache.spark.sql.sources._ - - // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` - // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be - // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - expression match { - case EqualTo(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .equals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case EqualNullSafe(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .nullSafeEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case LessThan(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .lessThan(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case LessThanOrEqual(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startAnd() - .lessThanEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case GreaterThan(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startNot() - .lessThanEquals(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case GreaterThanOrEqual(name, value) if dataTypeMap.contains(name) => - val castedValue = castLiteralValue(value, dataTypeMap(name).fieldType) - Some(builder.startNot() - .lessThan(dataTypeMap(name).fieldName, getType(name), castedValue).end()) - - case IsNull(name) if dataTypeMap.contains(name) => - Some(builder.startAnd().isNull(dataTypeMap(name).fieldName, getType(name)).end()) - - case IsNotNull(name) if dataTypeMap.contains(name) => - Some(builder.startNot().isNull(dataTypeMap(name).fieldName, getType(name)).end()) - - case In(name, values) if dataTypeMap.contains(name) => - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name).fieldType)) - Some(builder.startAnd().in(dataTypeMap(name).fieldName, getType(name), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) - - case _ => None - } - } -} - diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala deleted file mode 100644 index 7fbc1cd205b13..0000000000000 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.spark.sql.execution.datasources.orc - -import org.apache.orc.storage.common.`type`.HiveDecimal -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch -import org.apache.orc.storage.ql.io.sarg.{SearchArgument => OrcSearchArgument} -import org.apache.orc.storage.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} -import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} - -import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.types.Decimal - -/** - * Various utilities for ORC used to upgrade the built-in Hive. - */ -private[sql] object OrcShimUtils { - - class VectorizedRowBatchWrap(val batch: VectorizedRowBatch) {} - - private[sql] type Operator = OrcOperator - private[sql] type SearchArgument = OrcSearchArgument - - def getGregorianDays(value: Any): Int = { - new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays - } - - def getDecimal(value: Any): Decimal = { - val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() - Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) - } - - def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { - if (reuseObj) { - val result = new DaysWritable() - (getter, ordinal) => - result.set(getter.getInt(ordinal)) - result - } else { - (getter: SpecializedGetters, ordinal: Int) => - new DaysWritable(getter.getInt(ordinal)) - } - } - - def getHiveDecimalWritable(precision: Int, scale: Int): - (SpecializedGetters, Int) => HiveDecimalWritable = { - (getter, ordinal) => - val d = getter.getDecimal(ordinal, precision, scale) - new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) - } -} diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala deleted file mode 100644 index e159a0588dfff..0000000000000 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ /dev/null @@ -1,676 +0,0 @@ -/* - * 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.spark.sql.execution.datasources.orc - -import java.math.MathContext -import java.nio.charset.StandardCharsets -import java.sql.{Date, Timestamp} - -import scala.collection.JavaConverters._ - -import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Row} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types._ - -/** - * A test suite that tests Apache ORC filter API based filter pushdown optimization. - * OrcFilterSuite and HiveOrcFilterSuite is logically duplicated to provide the same test coverage. - * The difference are the packages containing 'Predicate' and 'SearchArgument' classes. - * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. - * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. - */ -class OrcFilterSuite extends OrcTest with SharedSparkSession { - - override protected def sparkConf: SparkConf = - super - .sparkConf - .set(SQLConf.USE_V1_SOURCE_LIST, "") - - protected def checkFilterPredicate( - df: DataFrame, - predicate: Predicate, - checker: (SearchArgument) => Unit): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - query.queryExecution.optimizedPlan match { - case PhysicalOperation(_, filters, DataSourceV2ScanRelation(_, o: OrcScan, _)) => - assert(filters.nonEmpty, "No filter is analyzed from the given query") - assert(o.pushedFilters.nonEmpty, "No filter is pushed down") - val maybeFilter = OrcFilters.createFilter(query.schema, o.pushedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for ${o.pushedFilters}") - checker(maybeFilter.get) - - case _ => - throw new AnalysisException("Can not match OrcTable in the query.") - } - } - - protected def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument) = { - val operator = filter.getLeaves.asScala - assert(operator.map(_.getOperator).contains(filterOperator)) - } - checkFilterPredicate(df, predicate, checkComparisonOperator) - } - - protected def checkFilterPredicate - (predicate: Predicate, stringExpr: String) - (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument) = { - assert(filter.toString == stringExpr) - } - checkFilterPredicate(df, predicate, checkLogicalOperator) - } - - test("filter pushdown - integer") { - withNestedOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val intAttr = df(colName).expr - assert(df(colName).expr.dataType === IntegerType) - - checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - long") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toLong)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val longAttr = df(colName).expr - assert(df(colName).expr.dataType === LongType) - - checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - float") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toFloat)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val floatAttr = df(colName).expr - assert(df(colName).expr.dataType === FloatType) - - checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - double") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val doubleAttr = df(colName).expr - assert(df(colName).expr.dataType === DoubleType) - - checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - string") { - withNestedOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val strAttr = df(colName).expr - assert(df(colName).expr.dataType === StringType) - - checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - boolean") { - withNestedOrcDataFrame( - (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val booleanAttr = df(colName).expr - assert(df(colName).expr.dataType === BooleanType) - - checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> booleanAttr, - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - decimal") { - withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => - implicit val df: DataFrame = inputDF - - val decimalAttr = df(colName).expr - assert(df(colName).expr.dataType === DecimalType(38, 18)) - - checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) - } - } - - test("filter pushdown - timestamp") { - val input = Seq( - "1000-01-01 01:02:03", - "1582-10-01 00:11:22", - "1900-01-01 23:59:59", - "2020-05-25 10:11:12").map(Timestamp.valueOf) - - withOrcFile(input.map(Tuple1(_))) { path => - Seq(false, true).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - readFile(path) { implicit df => - val timestamps = input.map(Literal(_)) - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(timestamps(2)) < $"_1", - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(timestamps(0)) >= $"_1", - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - } - } - } - - test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate( - $"_1".isNotNull, - "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" - ) - checkFilterPredicate( - $"_1" =!= 1, - "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" - ) - checkFilterPredicate( - !($"_1" < 4), - "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" - ) - checkFilterPredicate( - $"_1" < 2 || $"_1" > 3, - "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + - "expr = (or leaf-0 (not leaf-1))" - ) - checkFilterPredicate( - $"_1" < 2 && $"_1" > 3, - "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + - "expr = (and (not leaf-0) leaf-1 (not leaf-2))" - ) - } - } - - test("filter pushdown - date") { - val input = Seq("2017-08-18", "2017-08-19", "2017-08-20", "2017-08-21").map { day => - Date.valueOf(day) - } - withOrcFile(input.map(Tuple1(_))) { path => - Seq(false, true).foreach { java8Api => - withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString) { - readFile(path) { implicit df => - val dates = input.map(Literal(_)) - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === dates(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < dates(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= dates(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(dates(0) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(dates(0) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(dates(1) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(dates(2) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(dates(0) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(dates(3) <= $"_1", PredicateLeaf.Operator.LESS_THAN) - } - } - } - } - } - - test("no filter pushdown - non-supported types") { - implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) - } - // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull, noneSupported = true) - } - // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) - } - // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) - } - } - - test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - StringContains("b", "prefix") - )).get.toString - } - - // The `LessThan` should be converted while the whole inner `And` shouldn't - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan("a", 10), - Not(And( - GreaterThan("a", 1), - StringContains("b", "prefix") - )) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate and push down `LessThan` - assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ) - )).get.toString - } - - // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. - assertResult("leaf-0 = (LESS_THAN a 10), leaf-1 = (LESS_THAN_EQUALS a 1)," + - " expr = (and leaf-0 (not leaf-1))") { - OrcFilters.createFilter(schema, Array( - And( - And( - LessThan("a", 10), - StringContains("b", "prefix") - ), - GreaterThan("a", 1) - ) - )).get.toString - } - } - - test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { - import org.apache.spark.sql.sources._ - // The `LessThan` should be converted while the `StringContains` shouldn't - val schema = new StructType( - Array( - StructField("a", IntegerType, nullable = true), - StructField("b", StringType, nullable = true))) - - // The predicate `StringContains` predicate is not able to be pushed down. - assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + - " expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - GreaterThan("a", 10), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + - " expr = (or (not leaf-0) leaf-1)") { - OrcFilters.createFilter(schema, Array( - Or( - And( - GreaterThan("a", 10), - StringContains("b", "foobar") - ), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).get.toString - } - - assert(OrcFilters.createFilter(schema, Array( - Or( - StringContains("b", "foobar"), - And( - StringContains("b", "prefix"), - LessThan("a", 1) - ) - ) - )).isEmpty) - } - - test("SPARK-27160: Fix casting of the DecimalType literal") { - import org.apache.spark.sql.sources._ - val schema = StructType(Array(StructField("a", DecimalType(3, 2)))) - assertResult("leaf-0 = (LESS_THAN a 3.14), expr = leaf-0") { - OrcFilters.createFilter(schema, Array( - LessThan( - "a", - new java.math.BigDecimal(3.14, MathContext.DECIMAL64).setScale(2))) - ).get.toString - } - } - - test("SPARK-32622: case sensitivity in predicate pushdown") { - withTempPath { dir => - val count = 10 - val tableName = "spark_32622" - val tableDir1 = dir.getAbsoluteFile + "/table1" - - // Physical ORC files have both `A` and `a` fields. - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - spark.range(count).repartition(count).selectExpr("id - 1 as A", "id as a") - .write.mode("overwrite").orc(tableDir1) - } - - // Metastore table has both `A` and `a` fields too. - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - sql( - s""" - |CREATE TABLE $tableName (A LONG, a LONG) USING ORC LOCATION '$tableDir1' - """.stripMargin) - - checkAnswer(sql(s"select a, A from $tableName"), (0 until count).map(c => Row(c, c - 1))) - - val actual1 = stripSparkFilter(sql(s"select A from $tableName where A < 0")) - assert(actual1.count() == 1) - - val actual2 = stripSparkFilter(sql(s"select A from $tableName where a < 0")) - assert(actual2.count() == 0) - } - - // Exception thrown for ambiguous case. - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - val e = intercept[AnalysisException] { - sql(s"select a from $tableName where a < 0").collect() - } - assert(e.getMessage.contains( - "Reference 'a' is ambiguous")) - } - } - - // Metastore table has only `A` field. - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - sql( - s""" - |CREATE TABLE $tableName (A LONG) USING ORC LOCATION '$tableDir1' - """.stripMargin) - - val e = intercept[SparkException] { - sql(s"select A from $tableName where A < 0").collect() - } - assert(e.getCause.isInstanceOf[RuntimeException] && e.getCause.getMessage.contains( - """Found duplicate field(s) "A": [A, a] in case-insensitive mode""")) - } - } - - // Physical ORC files have only `A` field. - val tableDir2 = dir.getAbsoluteFile + "/table2" - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - spark.range(count).repartition(count).selectExpr("id - 1 as A") - .write.mode("overwrite").orc(tableDir2) - } - - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - sql( - s""" - |CREATE TABLE $tableName (a LONG) USING ORC LOCATION '$tableDir2' - """.stripMargin) - - checkAnswer(sql(s"select a from $tableName"), (0 until count).map(c => Row(c - 1))) - - val actual = stripSparkFilter(sql(s"select a from $tableName where a < 0")) - assert(actual.count() == 1) - } - } - - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - sql( - s""" - |CREATE TABLE $tableName (A LONG) USING ORC LOCATION '$tableDir2' - """.stripMargin) - - checkAnswer(sql(s"select A from $tableName"), (0 until count).map(c => Row(c - 1))) - - val actual = stripSparkFilter(sql(s"select A from $tableName where A < 0")) - assert(actual.count() == 1) - } - } - } - } - - test("SPARK-32646: Case-insensitive field resolution for pushdown when reading ORC") { - import org.apache.spark.sql.sources._ - - def getOrcFilter( - schema: StructType, - filters: Seq[Filter], - caseSensitive: String): Option[SearchArgument] = { - var orcFilter: Option[SearchArgument] = None - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { - orcFilter = - OrcFilters.createFilter(schema, filters) - } - orcFilter - } - - def testFilter( - schema: StructType, - filters: Seq[Filter], - expected: SearchArgument): Unit = { - val caseSensitiveFilters = getOrcFilter(schema, filters, "true") - val caseInsensitiveFilters = getOrcFilter(schema, filters, "false") - - assert(caseSensitiveFilters.isEmpty) - assert(caseInsensitiveFilters.isDefined) - - assert(caseInsensitiveFilters.get.getLeaves().size() > 0) - assert(caseInsensitiveFilters.get.getLeaves().size() == expected.getLeaves().size()) - (0 until expected.getLeaves().size()).foreach { index => - assert(caseInsensitiveFilters.get.getLeaves().get(index) == expected.getLeaves().get(index)) - } - } - - val schema1 = StructType(Seq(StructField("cint", IntegerType))) - testFilter(schema1, Seq(GreaterThan("CINT", 1)), - newBuilder.startNot() - .lessThanEquals("cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema1, Seq( - And(GreaterThan("CINT", 1), EqualTo("Cint", 2))), - newBuilder.startAnd() - .startNot() - .lessThanEquals("cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`() - .equals("cint", OrcFilters.getPredicateLeafType(IntegerType), 2L) - .`end`().build()) - - // Nested column case - val schema2 = StructType(Seq(StructField("a", - StructType(Seq(StructField("cint", IntegerType)))))) - - testFilter(schema2, Seq(GreaterThan("A.CINT", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq(GreaterThan("a.CINT", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq(GreaterThan("A.cint", 1)), - newBuilder.startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`().build()) - testFilter(schema2, Seq( - And(GreaterThan("a.CINT", 1), EqualTo("a.Cint", 2))), - newBuilder.startAnd() - .startNot() - .lessThanEquals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 1L).`end`() - .equals("a.cint", OrcFilters.getPredicateLeafType(IntegerType), 2L) - .`end`().build()) - } -} - diff --git a/sql/hive-thriftserver/v2.3/if/TCLIService.thrift b/sql/hive-thriftserver/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v2.3/if/TCLIService.thrift rename to sql/hive-thriftserver/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5bf20b209aff7..4a96afe9df20a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -146,9 +146,7 @@ - v${hive.version.short}/src/gen/java - v${hive.version.short}/src/main/java - v${hive.version.short}/src/main/scala + src/gen/java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala similarity index 100% rename from sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala diff --git a/sql/hive-thriftserver/v1.2/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2/if/TCLIService.thrift deleted file mode 100644 index 225e319737811..0000000000000 --- a/sql/hive-thriftserver/v1.2/if/TCLIService.thrift +++ /dev/null @@ -1,1173 +0,0 @@ -// 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. - -// Coding Conventions for this file: -// -// Structs/Enums/Unions -// * Struct, Enum, and Union names begin with a "T", -// and use a capital letter for each new word, with no underscores. -// * All fields should be declared as either optional or required. -// -// Functions -// * Function names start with a capital letter and have a capital letter for -// each new word, with no underscores. -// * Each function should take exactly one parameter, named TFunctionNameReq, -// and should return either void or TFunctionNameResp. This convention allows -// incremental updates. -// -// Services -// * Service names begin with the letter "T", use a capital letter for each -// new word (with no underscores), and end with the word "Service". - -namespace java org.apache.hive.service.cli.thrift -namespace cpp apache.hive.service.cli.thrift - -// List of protocol versions. A new token should be -// added to the end of this list every time a change is made. -enum TProtocolVersion { - HIVE_CLI_SERVICE_PROTOCOL_V1, - - // V2 adds support for asynchronous execution - HIVE_CLI_SERVICE_PROTOCOL_V2 - - // V3 add varchar type, primitive type qualifiers - HIVE_CLI_SERVICE_PROTOCOL_V3 - - // V4 add decimal precision/scale, char type - HIVE_CLI_SERVICE_PROTOCOL_V4 - - // V5 adds error details when GetOperationStatus returns in error state - HIVE_CLI_SERVICE_PROTOCOL_V5 - - // V6 uses binary type for binary payload (was string) and uses columnar result set - HIVE_CLI_SERVICE_PROTOCOL_V6 - - // V7 adds support for delegation token based connection - HIVE_CLI_SERVICE_PROTOCOL_V7 - - // V8 adds support for interval types - HIVE_CLI_SERVICE_PROTOCOL_V8 -} - -enum TTypeId { - BOOLEAN_TYPE, - TINYINT_TYPE, - SMALLINT_TYPE, - INT_TYPE, - BIGINT_TYPE, - FLOAT_TYPE, - DOUBLE_TYPE, - STRING_TYPE, - TIMESTAMP_TYPE, - BINARY_TYPE, - ARRAY_TYPE, - MAP_TYPE, - STRUCT_TYPE, - UNION_TYPE, - USER_DEFINED_TYPE, - DECIMAL_TYPE, - NULL_TYPE, - DATE_TYPE, - VARCHAR_TYPE, - CHAR_TYPE, - INTERVAL_YEAR_MONTH_TYPE, - INTERVAL_DAY_TIME_TYPE -} - -const set PRIMITIVE_TYPES = [ - TTypeId.BOOLEAN_TYPE, - TTypeId.TINYINT_TYPE, - TTypeId.SMALLINT_TYPE, - TTypeId.INT_TYPE, - TTypeId.BIGINT_TYPE, - TTypeId.FLOAT_TYPE, - TTypeId.DOUBLE_TYPE, - TTypeId.STRING_TYPE, - TTypeId.TIMESTAMP_TYPE, - TTypeId.BINARY_TYPE, - TTypeId.DECIMAL_TYPE, - TTypeId.NULL_TYPE, - TTypeId.DATE_TYPE, - TTypeId.VARCHAR_TYPE, - TTypeId.CHAR_TYPE, - TTypeId.INTERVAL_YEAR_MONTH_TYPE, - TTypeId.INTERVAL_DAY_TIME_TYPE -] - -const set COMPLEX_TYPES = [ - TTypeId.ARRAY_TYPE - TTypeId.MAP_TYPE - TTypeId.STRUCT_TYPE - TTypeId.UNION_TYPE - TTypeId.USER_DEFINED_TYPE -] - -const set COLLECTION_TYPES = [ - TTypeId.ARRAY_TYPE - TTypeId.MAP_TYPE -] - -const map TYPE_NAMES = { - TTypeId.BOOLEAN_TYPE: "BOOLEAN", - TTypeId.TINYINT_TYPE: "TINYINT", - TTypeId.SMALLINT_TYPE: "SMALLINT", - TTypeId.INT_TYPE: "INT", - TTypeId.BIGINT_TYPE: "BIGINT", - TTypeId.FLOAT_TYPE: "FLOAT", - TTypeId.DOUBLE_TYPE: "DOUBLE", - TTypeId.STRING_TYPE: "STRING", - TTypeId.TIMESTAMP_TYPE: "TIMESTAMP", - TTypeId.BINARY_TYPE: "BINARY", - TTypeId.ARRAY_TYPE: "ARRAY", - TTypeId.MAP_TYPE: "MAP", - TTypeId.STRUCT_TYPE: "STRUCT", - TTypeId.UNION_TYPE: "UNIONTYPE", - TTypeId.DECIMAL_TYPE: "DECIMAL", - TTypeId.NULL_TYPE: "NULL" - TTypeId.DATE_TYPE: "DATE" - TTypeId.VARCHAR_TYPE: "VARCHAR" - TTypeId.CHAR_TYPE: "CHAR" - TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH" - TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME" -} - -// Thrift does not support recursively defined types or forward declarations, -// which makes it difficult to represent Hive's nested types. -// To get around these limitations TTypeDesc employs a type list that maps -// integer "pointers" to TTypeEntry objects. The following examples show -// how different types are represented using this scheme: -// -// "INT": -// TTypeDesc { -// types = [ -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// } -// ] -// } -// -// "ARRAY": -// TTypeDesc { -// types = [ -// TTypeEntry.array_entry { -// object_type_ptr = 1 -// }, -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// } -// ] -// } -// -// "MAP": -// TTypeDesc { -// types = [ -// TTypeEntry.map_entry { -// key_type_ptr = 1 -// value_type_ptr = 2 -// }, -// TTypeEntry.primitive_entry { -// type = INT_TYPE -// }, -// TTypeEntry.primitive_entry { -// type = STRING_TYPE -// } -// ] -// } - -typedef i32 TTypeEntryPtr - -// Valid TTypeQualifiers key names -const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength" - -// Type qualifier key name for decimal -const string PRECISION = "precision" -const string SCALE = "scale" - -union TTypeQualifierValue { - 1: optional i32 i32Value - 2: optional string stringValue -} - -// Type qualifiers for primitive type. -struct TTypeQualifiers { - 1: required map qualifiers -} - -// Type entry for a primitive type. -struct TPrimitiveTypeEntry { - // The primitive type token. This must satisfy the condition - // that type is in the PRIMITIVE_TYPES set. - 1: required TTypeId type - 2: optional TTypeQualifiers typeQualifiers -} - -// Type entry for an ARRAY type. -struct TArrayTypeEntry { - 1: required TTypeEntryPtr objectTypePtr -} - -// Type entry for a MAP type. -struct TMapTypeEntry { - 1: required TTypeEntryPtr keyTypePtr - 2: required TTypeEntryPtr valueTypePtr -} - -// Type entry for a STRUCT type. -struct TStructTypeEntry { - 1: required map nameToTypePtr -} - -// Type entry for a UNIONTYPE type. -struct TUnionTypeEntry { - 1: required map nameToTypePtr -} - -struct TUserDefinedTypeEntry { - // The fully qualified name of the class implementing this type. - 1: required string typeClassName -} - -// We use a union here since Thrift does not support inheritance. -union TTypeEntry { - 1: TPrimitiveTypeEntry primitiveEntry - 2: TArrayTypeEntry arrayEntry - 3: TMapTypeEntry mapEntry - 4: TStructTypeEntry structEntry - 5: TUnionTypeEntry unionEntry - 6: TUserDefinedTypeEntry userDefinedTypeEntry -} - -// Type descriptor for columns. -struct TTypeDesc { - // The "top" type is always the first element of the list. - // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE - // type, then subsequent elements represent nested types. - 1: required list types -} - -// A result set column descriptor. -struct TColumnDesc { - // The name of the column - 1: required string columnName - - // The type descriptor for this column - 2: required TTypeDesc typeDesc - - // The ordinal position of this column in the schema - 3: required i32 position - - 4: optional string comment -} - -// Metadata used to describe the schema (column names, types, comments) -// of result sets. -struct TTableSchema { - 1: required list columns -} - -// A Boolean column value. -struct TBoolValue { - // NULL if value is unset. - 1: optional bool value -} - -// A Byte column value. -struct TByteValue { - // NULL if value is unset. - 1: optional byte value -} - -// A signed, 16 bit column value. -struct TI16Value { - // NULL if value is unset - 1: optional i16 value -} - -// A signed, 32 bit column value -struct TI32Value { - // NULL if value is unset - 1: optional i32 value -} - -// A signed 64 bit column value -struct TI64Value { - // NULL if value is unset - 1: optional i64 value -} - -// A floating point 64 bit column value -struct TDoubleValue { - // NULL if value is unset - 1: optional double value -} - -struct TStringValue { - // NULL if value is unset - 1: optional string value -} - -// A single column value in a result set. -// Note that Hive's type system is richer than Thrift's, -// so in some cases we have to map multiple Hive types -// to the same Thrift type. On the client-side this is -// disambiguated by looking at the Schema of the -// result set. -union TColumnValue { - 1: TBoolValue boolVal // BOOLEAN - 2: TByteValue byteVal // TINYINT - 3: TI16Value i16Val // SMALLINT - 4: TI32Value i32Val // INT - 5: TI64Value i64Val // BIGINT, TIMESTAMP - 6: TDoubleValue doubleVal // FLOAT, DOUBLE - 7: TStringValue stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME -} - -// Represents a row in a rowset. -struct TRow { - 1: required list colVals -} - -struct TBoolColumn { - 1: required list values - 2: required binary nulls -} - -struct TByteColumn { - 1: required list values - 2: required binary nulls -} - -struct TI16Column { - 1: required list values - 2: required binary nulls -} - -struct TI32Column { - 1: required list values - 2: required binary nulls -} - -struct TI64Column { - 1: required list values - 2: required binary nulls -} - -struct TDoubleColumn { - 1: required list values - 2: required binary nulls -} - -struct TStringColumn { - 1: required list values - 2: required binary nulls -} - -struct TBinaryColumn { - 1: required list values - 2: required binary nulls -} - -// Note that Hive's type system is richer than Thrift's, -// so in some cases we have to map multiple Hive types -// to the same Thrift type. On the client-side this is -// disambiguated by looking at the Schema of the -// result set. -union TColumn { - 1: TBoolColumn boolVal // BOOLEAN - 2: TByteColumn byteVal // TINYINT - 3: TI16Column i16Val // SMALLINT - 4: TI32Column i32Val // INT - 5: TI64Column i64Val // BIGINT, TIMESTAMP - 6: TDoubleColumn doubleVal // FLOAT, DOUBLE - 7: TStringColumn stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL - 8: TBinaryColumn binaryVal // BINARY -} - -// Represents a rowset -struct TRowSet { - // The starting row offset of this rowset. - 1: required i64 startRowOffset - 2: required list rows - 3: optional list columns -} - -// The return status code contained in each response. -enum TStatusCode { - SUCCESS_STATUS, - SUCCESS_WITH_INFO_STATUS, - STILL_EXECUTING_STATUS, - ERROR_STATUS, - INVALID_HANDLE_STATUS -} - -// The return status of a remote request -struct TStatus { - 1: required TStatusCode statusCode - - // If status is SUCCESS_WITH_INFO, info_msgs may be populated with - // additional diagnostic information. - 2: optional list infoMessages - - // If status is ERROR, then the following fields may be set - 3: optional string sqlState // as defined in the ISO/IEF CLI specification - 4: optional i32 errorCode // internal error code - 5: optional string errorMessage -} - -// The state of an operation (i.e. a query or other -// asynchronous operation that generates a result set) -// on the server. -enum TOperationState { - // The operation has been initialized - INITIALIZED_STATE, - - // The operation is running. In this state the result - // set is not available. - RUNNING_STATE, - - // The operation has completed. When an operation is in - // this state its result set may be fetched. - FINISHED_STATE, - - // The operation was canceled by a client - CANCELED_STATE, - - // The operation was closed by a client - CLOSED_STATE, - - // The operation failed due to an error - ERROR_STATE, - - // The operation is in an unrecognized state - UKNOWN_STATE, - - // The operation is in an pending state - PENDING_STATE, -} - -// A string identifier. This is interpreted literally. -typedef string TIdentifier - -// A search pattern. -// -// Valid search pattern characters: -// '_': Any single character. -// '%': Any sequence of zero or more characters. -// '\': Escape character used to include special characters, -// e.g. '_', '%', '\'. If a '\' precedes a non-special -// character it has no special meaning and is interpreted -// literally. -typedef string TPattern - - -// A search pattern or identifier. Used as input -// parameter for many of the catalog functions. -typedef string TPatternOrIdentifier - -struct THandleIdentifier { - // 16 byte globally unique identifier - // This is the public ID of the handle and - // can be used for reporting. - 1: required binary guid, - - // 16 byte secret generated by the server - // and used to verify that the handle is not - // being hijacked by another user. - 2: required binary secret, -} - -// Client-side handle to persistent -// session information on the server-side. -struct TSessionHandle { - 1: required THandleIdentifier sessionId -} - -// The subtype of an OperationHandle. -enum TOperationType { - EXECUTE_STATEMENT, - GET_TYPE_INFO, - GET_CATALOGS, - GET_SCHEMAS, - GET_TABLES, - GET_TABLE_TYPES, - GET_COLUMNS, - GET_FUNCTIONS, - UNKNOWN, -} - -// Client-side reference to a task running -// asynchronously on the server. -struct TOperationHandle { - 1: required THandleIdentifier operationId - 2: required TOperationType operationType - - // If hasResultSet = TRUE, then this operation - // generates a result set that can be fetched. - // Note that the result set may be empty. - // - // If hasResultSet = FALSE, then this operation - // does not generate a result set, and calling - // GetResultSetMetadata or FetchResults against - // this OperationHandle will generate an error. - 3: required bool hasResultSet - - // For operations that don't generate result sets, - // modifiedRowCount is either: - // - // 1) The number of rows that were modified by - // the DML operation (e.g. number of rows inserted, - // number of rows deleted, etc). - // - // 2) 0 for operations that don't modify or add rows. - // - // 3) < 0 if the operation is capable of modifiying rows, - // but Hive is unable to determine how many rows were - // modified. For example, Hive's LOAD DATA command - // doesn't generate row count information because - // Hive doesn't inspect the data as it is loaded. - // - // modifiedRowCount is unset if the operation generates - // a result set. - 4: optional double modifiedRowCount -} - - -// OpenSession() -// -// Open a session (connection) on the server against -// which operations may be executed. -struct TOpenSessionReq { - // The version of the HiveServer2 protocol that the client is using. - 1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 - - // Username and password for authentication. - // Depending on the authentication scheme being used, - // this information may instead be provided by a lower - // protocol layer, in which case these fields may be - // left unset. - 2: optional string username - 3: optional string password - - // Configuration overlay which is applied when the session is - // first created. - 4: optional map configuration -} - -struct TOpenSessionResp { - 1: required TStatus status - - // The protocol version that the server is using. - 2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 - - // Session Handle - 3: optional TSessionHandle sessionHandle - - // The configuration settings for this session. - 4: optional map configuration -} - - -// CloseSession() -// -// Closes the specified session and frees any resources -// currently allocated to that session. Any open -// operations in that session will be canceled. -struct TCloseSessionReq { - 1: required TSessionHandle sessionHandle -} - -struct TCloseSessionResp { - 1: required TStatus status -} - - - -enum TGetInfoType { - CLI_MAX_DRIVER_CONNECTIONS = 0, - CLI_MAX_CONCURRENT_ACTIVITIES = 1, - CLI_DATA_SOURCE_NAME = 2, - CLI_FETCH_DIRECTION = 8, - CLI_SERVER_NAME = 13, - CLI_SEARCH_PATTERN_ESCAPE = 14, - CLI_DBMS_NAME = 17, - CLI_DBMS_VER = 18, - CLI_ACCESSIBLE_TABLES = 19, - CLI_ACCESSIBLE_PROCEDURES = 20, - CLI_CURSOR_COMMIT_BEHAVIOR = 23, - CLI_DATA_SOURCE_READ_ONLY = 25, - CLI_DEFAULT_TXN_ISOLATION = 26, - CLI_IDENTIFIER_CASE = 28, - CLI_IDENTIFIER_QUOTE_CHAR = 29, - CLI_MAX_COLUMN_NAME_LEN = 30, - CLI_MAX_CURSOR_NAME_LEN = 31, - CLI_MAX_SCHEMA_NAME_LEN = 32, - CLI_MAX_CATALOG_NAME_LEN = 34, - CLI_MAX_TABLE_NAME_LEN = 35, - CLI_SCROLL_CONCURRENCY = 43, - CLI_TXN_CAPABLE = 46, - CLI_USER_NAME = 47, - CLI_TXN_ISOLATION_OPTION = 72, - CLI_INTEGRITY = 73, - CLI_GETDATA_EXTENSIONS = 81, - CLI_NULL_COLLATION = 85, - CLI_ALTER_TABLE = 86, - CLI_ORDER_BY_COLUMNS_IN_SELECT = 90, - CLI_SPECIAL_CHARACTERS = 94, - CLI_MAX_COLUMNS_IN_GROUP_BY = 97, - CLI_MAX_COLUMNS_IN_INDEX = 98, - CLI_MAX_COLUMNS_IN_ORDER_BY = 99, - CLI_MAX_COLUMNS_IN_SELECT = 100, - CLI_MAX_COLUMNS_IN_TABLE = 101, - CLI_MAX_INDEX_SIZE = 102, - CLI_MAX_ROW_SIZE = 104, - CLI_MAX_STATEMENT_LEN = 105, - CLI_MAX_TABLES_IN_SELECT = 106, - CLI_MAX_USER_NAME_LEN = 107, - CLI_OJ_CAPABILITIES = 115, - - CLI_XOPEN_CLI_YEAR = 10000, - CLI_CURSOR_SENSITIVITY = 10001, - CLI_DESCRIBE_PARAMETER = 10002, - CLI_CATALOG_NAME = 10003, - CLI_COLLATION_SEQ = 10004, - CLI_MAX_IDENTIFIER_LEN = 10005, -} - -union TGetInfoValue { - 1: string stringValue - 2: i16 smallIntValue - 3: i32 integerBitmask - 4: i32 integerFlag - 5: i32 binaryValue - 6: i64 lenValue -} - -// GetInfo() -// -// This function is based on ODBC's CLIGetInfo() function. -// The function returns general information about the data source -// using the same keys as ODBC. -struct TGetInfoReq { - // The session to run this request against - 1: required TSessionHandle sessionHandle - - 2: required TGetInfoType infoType -} - -struct TGetInfoResp { - 1: required TStatus status - - 2: required TGetInfoValue infoValue -} - - -// ExecuteStatement() -// -// Execute a statement. -// The returned OperationHandle can be used to check on the -// status of the statement, and to fetch results once the -// statement has finished executing. -struct TExecuteStatementReq { - // The session to execute the statement against - 1: required TSessionHandle sessionHandle - - // The statement to be executed (DML, DDL, SET, etc) - 2: required string statement - - // Configuration properties that are overlayed on top of the - // the existing session configuration before this statement - // is executed. These properties apply to this statement - // only and will not affect the subsequent state of the Session. - 3: optional map confOverlay - - // Execute asynchronously when runAsync is true - 4: optional bool runAsync = false -} - -struct TExecuteStatementResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - -// GetTypeInfo() -// -// Get information about types supported by the HiveServer instance. -// The information is returned as a result set which can be fetched -// using the OperationHandle provided in the response. -// -// Refer to the documentation for ODBC's CLIGetTypeInfo function for -// the format of the result set. -struct TGetTypeInfoReq { - // The session to run this request against. - 1: required TSessionHandle sessionHandle -} - -struct TGetTypeInfoResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetCatalogs() -// -// Returns the list of catalogs (databases) -// Results are ordered by TABLE_CATALOG -// -// Resultset columns : -// col1 -// name: TABLE_CAT -// type: STRING -// desc: Catalog name. NULL if not applicable. -// -struct TGetCatalogsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle -} - -struct TGetCatalogsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetSchemas() -// -// Retrieves the schema names available in this database. -// The results are ordered by TABLE_CATALOG and TABLE_SCHEM. -// col1 -// name: TABLE_SCHEM -// type: STRING -// desc: schema name -// col2 -// name: TABLE_CATALOG -// type: STRING -// desc: catalog name -struct TGetSchemasReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog. Must not contain a search pattern. - 2: optional TIdentifier catalogName - - // schema name or pattern - 3: optional TPatternOrIdentifier schemaName -} - -struct TGetSchemasResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetTables() -// -// Returns a list of tables with catalog, schema, and table -// type information. The information is returned as a result -// set which can be fetched using the OperationHandle -// provided in the response. -// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME -// -// Result Set Columns: -// -// col1 -// name: TABLE_CAT -// type: STRING -// desc: Catalog name. NULL if not applicable. -// -// col2 -// name: TABLE_SCHEM -// type: STRING -// desc: Schema name. -// -// col3 -// name: TABLE_NAME -// type: STRING -// desc: Table name. -// -// col4 -// name: TABLE_TYPE -// type: STRING -// desc: The table type, e.g. "TABLE", "VIEW", etc. -// -// col5 -// name: REMARKS -// type: STRING -// desc: Comments about the table -// -struct TGetTablesReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog or a search pattern. - 2: optional TPatternOrIdentifier catalogName - - // Name of the schema or a search pattern. - 3: optional TPatternOrIdentifier schemaName - - // Name of the table or a search pattern. - 4: optional TPatternOrIdentifier tableName - - // List of table types to match - // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", - // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc. - 5: optional list tableTypes -} - -struct TGetTablesResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetTableTypes() -// -// Returns the table types available in this database. -// The results are ordered by table type. -// -// col1 -// name: TABLE_TYPE -// type: STRING -// desc: Table type name. -struct TGetTableTypesReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle -} - -struct TGetTableTypesResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetColumns() -// -// Returns a list of columns in the specified tables. -// The information is returned as a result set which can be fetched -// using the OperationHandle provided in the response. -// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, -// and ORDINAL_POSITION. -// -// Result Set Columns are the same as those for the ODBC CLIColumns -// function. -// -struct TGetColumnsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // Name of the catalog. Must not contain a search pattern. - 2: optional TIdentifier catalogName - - // Schema name or search pattern - 3: optional TPatternOrIdentifier schemaName - - // Table name or search pattern - 4: optional TPatternOrIdentifier tableName - - // Column name or search pattern - 5: optional TPatternOrIdentifier columnName -} - -struct TGetColumnsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetFunctions() -// -// Returns a list of functions supported by the data source. The -// behavior of this function matches -// java.sql.DatabaseMetaData.getFunctions() both in terms of -// inputs and outputs. -// -// Result Set Columns: -// -// col1 -// name: FUNCTION_CAT -// type: STRING -// desc: Function catalog (may be null) -// -// col2 -// name: FUNCTION_SCHEM -// type: STRING -// desc: Function schema (may be null) -// -// col3 -// name: FUNCTION_NAME -// type: STRING -// desc: Function name. This is the name used to invoke the function. -// -// col4 -// name: REMARKS -// type: STRING -// desc: Explanatory comment on the function. -// -// col5 -// name: FUNCTION_TYPE -// type: SMALLINT -// desc: Kind of function. One of: -// * functionResultUnknown - Cannot determine if a return value or a table -// will be returned. -// * functionNoTable - Does not a return a table. -// * functionReturnsTable - Returns a table. -// -// col6 -// name: SPECIFIC_NAME -// type: STRING -// desc: The name which uniquely identifies this function within its schema. -// In this case this is the fully qualified class name of the class -// that implements this function. -// -struct TGetFunctionsReq { - // Session to run this request against - 1: required TSessionHandle sessionHandle - - // A catalog name; must match the catalog name as it is stored in the - // database; "" retrieves those without a catalog; null means - // that the catalog name should not be used to narrow the search. - 2: optional TIdentifier catalogName - - // A schema name pattern; must match the schema name as it is stored - // in the database; "" retrieves those without a schema; null means - // that the schema name should not be used to narrow the search. - 3: optional TPatternOrIdentifier schemaName - - // A function name pattern; must match the function name as it is stored - // in the database. - 4: required TPatternOrIdentifier functionName -} - -struct TGetFunctionsResp { - 1: required TStatus status - 2: optional TOperationHandle operationHandle -} - - -// GetOperationStatus() -// -// Get the status of an operation running on the server. -struct TGetOperationStatusReq { - // Session to run this request against - 1: required TOperationHandle operationHandle -} - -struct TGetOperationStatusResp { - 1: required TStatus status - 2: optional TOperationState operationState - - // If operationState is ERROR_STATE, then the following fields may be set - // sqlState as defined in the ISO/IEF CLI specification - 3: optional string sqlState - - // Internal error code - 4: optional i32 errorCode - - // Error message - 5: optional string errorMessage -} - - -// CancelOperation() -// -// Cancels processing on the specified operation handle and -// frees any resources which were allocated. -struct TCancelOperationReq { - // Operation to cancel - 1: required TOperationHandle operationHandle -} - -struct TCancelOperationResp { - 1: required TStatus status -} - - -// CloseOperation() -// -// Given an operation in the FINISHED, CANCELED, -// or ERROR states, CloseOperation() will free -// all of the resources which were allocated on -// the server to service the operation. -struct TCloseOperationReq { - 1: required TOperationHandle operationHandle -} - -struct TCloseOperationResp { - 1: required TStatus status -} - - -// GetResultSetMetadata() -// -// Retrieves schema information for the specified operation -struct TGetResultSetMetadataReq { - // Operation for which to fetch result set schema information - 1: required TOperationHandle operationHandle -} - -struct TGetResultSetMetadataResp { - 1: required TStatus status - 2: optional TTableSchema schema -} - - -enum TFetchOrientation { - // Get the next rowset. The fetch offset is ignored. - FETCH_NEXT, - - // Get the previous rowset. The fetch offset is ignored. - FETCH_PRIOR, - - // Return the rowset at the given fetch offset relative - // to the current rowset. - // NOT SUPPORTED - FETCH_RELATIVE, - - // Return the rowset at the specified fetch offset. - // NOT SUPPORTED - FETCH_ABSOLUTE, - - // Get the first rowset in the result set. - FETCH_FIRST, - - // Get the last rowset in the result set. - // NOT SUPPORTED - FETCH_LAST -} - -// FetchResults() -// -// Fetch rows from the server corresponding to -// a particular OperationHandle. -struct TFetchResultsReq { - // Operation from which to fetch results. - 1: required TOperationHandle operationHandle - - // The fetch orientation. This must be either - // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. - 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT - - // Max number of rows that should be returned in - // the rowset. - 3: required i64 maxRows - - // The type of a fetch results request. 0 represents Query output. 1 represents Log - 4: optional i16 fetchType = 0 -} - -struct TFetchResultsResp { - 1: required TStatus status - - // TRUE if there are more rows left to fetch from the server. - 2: optional bool hasMoreRows - - // The rowset. This is optional so that we have the - // option in the future of adding alternate formats for - // representing result set data, e.g. delimited strings, - // binary encoded, etc. - 3: optional TRowSet results -} - -// GetDelegationToken() -// Retrieve delegation token for the current user -struct TGetDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // userid for the proxy user - 2: required string owner - - // designated renewer userid - 3: required string renewer -} - -struct TGetDelegationTokenResp { - // status of the request - 1: required TStatus status - - // delegation token string - 2: optional string delegationToken -} - -// CancelDelegationToken() -// Cancel the given delegation token -struct TCancelDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // delegation token to cancel - 2: required string delegationToken -} - -struct TCancelDelegationTokenResp { - // status of the request - 1: required TStatus status -} - -// RenewDelegationToken() -// Renew the given delegation token -struct TRenewDelegationTokenReq { - // session handle - 1: required TSessionHandle sessionHandle - - // delegation token to renew - 2: required string delegationToken -} - -struct TRenewDelegationTokenResp { - // status of the request - 1: required TStatus status -} - -service TCLIService { - - TOpenSessionResp OpenSession(1:TOpenSessionReq req); - - TCloseSessionResp CloseSession(1:TCloseSessionReq req); - - TGetInfoResp GetInfo(1:TGetInfoReq req); - - TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req); - - TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req); - - TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req); - - TGetSchemasResp GetSchemas(1:TGetSchemasReq req); - - TGetTablesResp GetTables(1:TGetTablesReq req); - - TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req); - - TGetColumnsResp GetColumns(1:TGetColumnsReq req); - - TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req); - - TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req); - - TCancelOperationResp CancelOperation(1:TCancelOperationReq req); - - TCloseOperationResp CloseOperation(1:TCloseOperationReq req); - - TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req); - - TFetchResultsResp FetchResults(1:TFetchResultsReq req); - - TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req); - - TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req); - - TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req); -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java deleted file mode 100644 index 6323d34eac734..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java +++ /dev/null @@ -1,383 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TArrayTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry"); - - private static final org.apache.thrift.protocol.TField OBJECT_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("objectTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TArrayTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TArrayTypeEntryTupleSchemeFactory()); - } - - private int objectTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OBJECT_TYPE_PTR((short)1, "objectTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OBJECT_TYPE_PTR - return OBJECT_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __OBJECTTYPEPTR_ISSET_ID = 0; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OBJECT_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("objectTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TArrayTypeEntry.class, metaDataMap); - } - - public TArrayTypeEntry() { - } - - public TArrayTypeEntry( - int objectTypePtr) - { - this(); - this.objectTypePtr = objectTypePtr; - setObjectTypePtrIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TArrayTypeEntry(TArrayTypeEntry other) { - __isset_bitfield = other.__isset_bitfield; - this.objectTypePtr = other.objectTypePtr; - } - - public TArrayTypeEntry deepCopy() { - return new TArrayTypeEntry(this); - } - - @Override - public void clear() { - setObjectTypePtrIsSet(false); - this.objectTypePtr = 0; - } - - public int getObjectTypePtr() { - return this.objectTypePtr; - } - - public void setObjectTypePtr(int objectTypePtr) { - this.objectTypePtr = objectTypePtr; - setObjectTypePtrIsSet(true); - } - - public void unsetObjectTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); - } - - /** Returns true if field objectTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetObjectTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); - } - - public void setObjectTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OBJECT_TYPE_PTR: - if (value == null) { - unsetObjectTypePtr(); - } else { - setObjectTypePtr((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OBJECT_TYPE_PTR: - return Integer.valueOf(getObjectTypePtr()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OBJECT_TYPE_PTR: - return isSetObjectTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TArrayTypeEntry) - return this.equals((TArrayTypeEntry)that); - return false; - } - - public boolean equals(TArrayTypeEntry that) { - if (that == null) - return false; - - boolean this_present_objectTypePtr = true; - boolean that_present_objectTypePtr = true; - if (this_present_objectTypePtr || that_present_objectTypePtr) { - if (!(this_present_objectTypePtr && that_present_objectTypePtr)) - return false; - if (this.objectTypePtr != that.objectTypePtr) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_objectTypePtr = true; - builder.append(present_objectTypePtr); - if (present_objectTypePtr) - builder.append(objectTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TArrayTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TArrayTypeEntry typedOther = (TArrayTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetObjectTypePtr()).compareTo(typedOther.isSetObjectTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetObjectTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.objectTypePtr, typedOther.objectTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TArrayTypeEntry("); - boolean first = true; - - sb.append("objectTypePtr:"); - sb.append(this.objectTypePtr); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetObjectTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'objectTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TArrayTypeEntryStandardSchemeFactory implements SchemeFactory { - public TArrayTypeEntryStandardScheme getScheme() { - return new TArrayTypeEntryStandardScheme(); - } - } - - private static class TArrayTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OBJECT_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.objectTypePtr = iprot.readI32(); - struct.setObjectTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(OBJECT_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.objectTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TArrayTypeEntryTupleSchemeFactory implements SchemeFactory { - public TArrayTypeEntryTupleScheme getScheme() { - return new TArrayTypeEntryTupleScheme(); - } - } - - private static class TArrayTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.objectTypePtr); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.objectTypePtr = iprot.readI32(); - struct.setObjectTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java deleted file mode 100644 index 6b1b054d1acad..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java +++ /dev/null @@ -1,550 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBinaryColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBinaryColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBinaryColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBinaryColumn.class, metaDataMap); - } - - public TBinaryColumn() { - } - - public TBinaryColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TBinaryColumn(TBinaryColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (ByteBuffer other_element : other.values) { - ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element); -; - __this__values.add(temp_binary_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TBinaryColumn deepCopy() { - return new TBinaryColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(ByteBuffer elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBinaryColumn) - return this.equals((TBinaryColumn)that); - return false; - } - - public boolean equals(TBinaryColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TBinaryColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBinaryColumn typedOther = (TBinaryColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBinaryColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBinaryColumnStandardSchemeFactory implements SchemeFactory { - public TBinaryColumnStandardScheme getScheme() { - return new TBinaryColumnStandardScheme(); - } - } - - private static class TBinaryColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBinaryColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list110 = iprot.readListBegin(); - struct.values = new ArrayList(_list110.size); - for (int _i111 = 0; _i111 < _list110.size; ++_i111) - { - ByteBuffer _elem112; // optional - _elem112 = iprot.readBinary(); - struct.values.add(_elem112); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBinaryColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); - for (ByteBuffer _iter113 : struct.values) - { - oprot.writeBinary(_iter113); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBinaryColumnTupleSchemeFactory implements SchemeFactory { - public TBinaryColumnTupleScheme getScheme() { - return new TBinaryColumnTupleScheme(); - } - } - - private static class TBinaryColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (ByteBuffer _iter114 : struct.values) - { - oprot.writeBinary(_iter114); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.values = new ArrayList(_list115.size); - for (int _i116 = 0; _i116 < _list115.size; ++_i116) - { - ByteBuffer _elem117; // optional - _elem117 = iprot.readBinary(); - struct.values.add(_elem117); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java deleted file mode 100644 index efd571cfdfbbf..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBoolColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBoolColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBoolColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolColumn.class, metaDataMap); - } - - public TBoolColumn() { - } - - public TBoolColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TBoolColumn(TBoolColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Boolean other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TBoolColumn deepCopy() { - return new TBoolColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(boolean elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBoolColumn) - return this.equals((TBoolColumn)that); - return false; - } - - public boolean equals(TBoolColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TBoolColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBoolColumn typedOther = (TBoolColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBoolColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBoolColumnStandardSchemeFactory implements SchemeFactory { - public TBoolColumnStandardScheme getScheme() { - return new TBoolColumnStandardScheme(); - } - } - - private static class TBoolColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); - struct.values = new ArrayList(_list54.size); - for (int _i55 = 0; _i55 < _list54.size; ++_i55) - { - boolean _elem56; // optional - _elem56 = iprot.readBool(); - struct.values.add(_elem56); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, struct.values.size())); - for (boolean _iter57 : struct.values) - { - oprot.writeBool(_iter57); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBoolColumnTupleSchemeFactory implements SchemeFactory { - public TBoolColumnTupleScheme getScheme() { - return new TBoolColumnTupleScheme(); - } - } - - private static class TBoolColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (boolean _iter58 : struct.values) - { - oprot.writeBool(_iter58); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list59 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, iprot.readI32()); - struct.values = new ArrayList(_list59.size); - for (int _i60 = 0; _i60 < _list59.size; ++_i60) - { - boolean _elem61; // optional - _elem61 = iprot.readBool(); - struct.values.add(_elem61); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java deleted file mode 100644 index c7495ee79e4b5..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TBoolValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BOOL, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TBoolValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TBoolValueTupleSchemeFactory()); - } - - private boolean value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolValue.class, metaDataMap); - } - - public TBoolValue() { - } - - /** - * Performs a deep copy on other. - */ - public TBoolValue(TBoolValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TBoolValue deepCopy() { - return new TBoolValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = false; - } - - public boolean isValue() { - return this.value; - } - - public void setValue(boolean value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Boolean.valueOf(isValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TBoolValue) - return this.equals((TBoolValue)that); - return false; - } - - public boolean equals(TBoolValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TBoolValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TBoolValue typedOther = (TBoolValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TBoolValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TBoolValueStandardSchemeFactory implements SchemeFactory { - public TBoolValueStandardScheme getScheme() { - return new TBoolValueStandardScheme(); - } - } - - private static class TBoolValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.value = iprot.readBool(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeBool(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TBoolValueTupleSchemeFactory implements SchemeFactory { - public TBoolValueTupleScheme getScheme() { - return new TBoolValueTupleScheme(); - } - } - - private static class TBoolValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeBool(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readBool(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java deleted file mode 100644 index 169bfdeab3eea..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TByteColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TByteColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TByteColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteColumn.class, metaDataMap); - } - - public TByteColumn() { - } - - public TByteColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TByteColumn(TByteColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Byte other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TByteColumn deepCopy() { - return new TByteColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(byte elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TByteColumn) - return this.equals((TByteColumn)that); - return false; - } - - public boolean equals(TByteColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TByteColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TByteColumn typedOther = (TByteColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TByteColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TByteColumnStandardSchemeFactory implements SchemeFactory { - public TByteColumnStandardScheme getScheme() { - return new TByteColumnStandardScheme(); - } - } - - private static class TByteColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TByteColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); - struct.values = new ArrayList(_list62.size); - for (int _i63 = 0; _i63 < _list62.size; ++_i63) - { - byte _elem64; // optional - _elem64 = iprot.readByte(); - struct.values.add(_elem64); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TByteColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, struct.values.size())); - for (byte _iter65 : struct.values) - { - oprot.writeByte(_iter65); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TByteColumnTupleSchemeFactory implements SchemeFactory { - public TByteColumnTupleScheme getScheme() { - return new TByteColumnTupleScheme(); - } - } - - private static class TByteColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (byte _iter66 : struct.values) - { - oprot.writeByte(_iter66); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, iprot.readI32()); - struct.values = new ArrayList(_list67.size); - for (int _i68 = 0; _i68 < _list67.size; ++_i68) - { - byte _elem69; // optional - _elem69 = iprot.readByte(); - struct.values.add(_elem69); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java deleted file mode 100644 index 23d9693759968..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TByteValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BYTE, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TByteValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TByteValueTupleSchemeFactory()); - } - - private byte value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteValue.class, metaDataMap); - } - - public TByteValue() { - } - - /** - * Performs a deep copy on other. - */ - public TByteValue(TByteValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TByteValue deepCopy() { - return new TByteValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public byte getValue() { - return this.value; - } - - public void setValue(byte value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Byte)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Byte.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TByteValue) - return this.equals((TByteValue)that); - return false; - } - - public boolean equals(TByteValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TByteValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TByteValue typedOther = (TByteValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TByteValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TByteValueStandardSchemeFactory implements SchemeFactory { - public TByteValueStandardScheme getScheme() { - return new TByteValueStandardScheme(); - } - } - - private static class TByteValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TByteValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { - struct.value = iprot.readByte(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TByteValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeByte(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TByteValueTupleSchemeFactory implements SchemeFactory { - public TByteValueTupleScheme getScheme() { - return new TByteValueTupleScheme(); - } - } - - private static class TByteValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeByte(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readByte(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java deleted file mode 100644 index 54851b8d51317..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java +++ /dev/null @@ -1,15414 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCLIService { - - public interface Iface { - - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException; - - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException; - - public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException; - - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException; - - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException; - - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException; - - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException; - - public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException; - - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException; - - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException; - - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException; - - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException; - - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException; - - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException; - - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException; - - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException; - - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException; - - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException; - - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException; - - } - - public interface AsyncIface { - - public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - } - - public static class Client extends org.apache.thrift.TServiceClient implements Iface { - public static class Factory implements org.apache.thrift.TServiceClientFactory { - public Factory() {} - public Client getClient(org.apache.thrift.protocol.TProtocol prot) { - return new Client(prot); - } - public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { - return new Client(iprot, oprot); - } - } - - public Client(org.apache.thrift.protocol.TProtocol prot) - { - super(prot, prot); - } - - public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { - super(iprot, oprot); - } - - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException - { - send_OpenSession(req); - return recv_OpenSession(); - } - - public void send_OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException - { - OpenSession_args args = new OpenSession_args(); - args.setReq(req); - sendBase("OpenSession", args); - } - - public TOpenSessionResp recv_OpenSession() throws org.apache.thrift.TException - { - OpenSession_result result = new OpenSession_result(); - receiveBase(result, "OpenSession"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "OpenSession failed: unknown result"); - } - - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException - { - send_CloseSession(req); - return recv_CloseSession(); - } - - public void send_CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException - { - CloseSession_args args = new CloseSession_args(); - args.setReq(req); - sendBase("CloseSession", args); - } - - public TCloseSessionResp recv_CloseSession() throws org.apache.thrift.TException - { - CloseSession_result result = new CloseSession_result(); - receiveBase(result, "CloseSession"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseSession failed: unknown result"); - } - - public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException - { - send_GetInfo(req); - return recv_GetInfo(); - } - - public void send_GetInfo(TGetInfoReq req) throws org.apache.thrift.TException - { - GetInfo_args args = new GetInfo_args(); - args.setReq(req); - sendBase("GetInfo", args); - } - - public TGetInfoResp recv_GetInfo() throws org.apache.thrift.TException - { - GetInfo_result result = new GetInfo_result(); - receiveBase(result, "GetInfo"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetInfo failed: unknown result"); - } - - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException - { - send_ExecuteStatement(req); - return recv_ExecuteStatement(); - } - - public void send_ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException - { - ExecuteStatement_args args = new ExecuteStatement_args(); - args.setReq(req); - sendBase("ExecuteStatement", args); - } - - public TExecuteStatementResp recv_ExecuteStatement() throws org.apache.thrift.TException - { - ExecuteStatement_result result = new ExecuteStatement_result(); - receiveBase(result, "ExecuteStatement"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "ExecuteStatement failed: unknown result"); - } - - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException - { - send_GetTypeInfo(req); - return recv_GetTypeInfo(); - } - - public void send_GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException - { - GetTypeInfo_args args = new GetTypeInfo_args(); - args.setReq(req); - sendBase("GetTypeInfo", args); - } - - public TGetTypeInfoResp recv_GetTypeInfo() throws org.apache.thrift.TException - { - GetTypeInfo_result result = new GetTypeInfo_result(); - receiveBase(result, "GetTypeInfo"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTypeInfo failed: unknown result"); - } - - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException - { - send_GetCatalogs(req); - return recv_GetCatalogs(); - } - - public void send_GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException - { - GetCatalogs_args args = new GetCatalogs_args(); - args.setReq(req); - sendBase("GetCatalogs", args); - } - - public TGetCatalogsResp recv_GetCatalogs() throws org.apache.thrift.TException - { - GetCatalogs_result result = new GetCatalogs_result(); - receiveBase(result, "GetCatalogs"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCatalogs failed: unknown result"); - } - - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException - { - send_GetSchemas(req); - return recv_GetSchemas(); - } - - public void send_GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException - { - GetSchemas_args args = new GetSchemas_args(); - args.setReq(req); - sendBase("GetSchemas", args); - } - - public TGetSchemasResp recv_GetSchemas() throws org.apache.thrift.TException - { - GetSchemas_result result = new GetSchemas_result(); - receiveBase(result, "GetSchemas"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetSchemas failed: unknown result"); - } - - public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException - { - send_GetTables(req); - return recv_GetTables(); - } - - public void send_GetTables(TGetTablesReq req) throws org.apache.thrift.TException - { - GetTables_args args = new GetTables_args(); - args.setReq(req); - sendBase("GetTables", args); - } - - public TGetTablesResp recv_GetTables() throws org.apache.thrift.TException - { - GetTables_result result = new GetTables_result(); - receiveBase(result, "GetTables"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTables failed: unknown result"); - } - - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException - { - send_GetTableTypes(req); - return recv_GetTableTypes(); - } - - public void send_GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException - { - GetTableTypes_args args = new GetTableTypes_args(); - args.setReq(req); - sendBase("GetTableTypes", args); - } - - public TGetTableTypesResp recv_GetTableTypes() throws org.apache.thrift.TException - { - GetTableTypes_result result = new GetTableTypes_result(); - receiveBase(result, "GetTableTypes"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTableTypes failed: unknown result"); - } - - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException - { - send_GetColumns(req); - return recv_GetColumns(); - } - - public void send_GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException - { - GetColumns_args args = new GetColumns_args(); - args.setReq(req); - sendBase("GetColumns", args); - } - - public TGetColumnsResp recv_GetColumns() throws org.apache.thrift.TException - { - GetColumns_result result = new GetColumns_result(); - receiveBase(result, "GetColumns"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetColumns failed: unknown result"); - } - - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException - { - send_GetFunctions(req); - return recv_GetFunctions(); - } - - public void send_GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException - { - GetFunctions_args args = new GetFunctions_args(); - args.setReq(req); - sendBase("GetFunctions", args); - } - - public TGetFunctionsResp recv_GetFunctions() throws org.apache.thrift.TException - { - GetFunctions_result result = new GetFunctions_result(); - receiveBase(result, "GetFunctions"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetFunctions failed: unknown result"); - } - - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException - { - send_GetOperationStatus(req); - return recv_GetOperationStatus(); - } - - public void send_GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException - { - GetOperationStatus_args args = new GetOperationStatus_args(); - args.setReq(req); - sendBase("GetOperationStatus", args); - } - - public TGetOperationStatusResp recv_GetOperationStatus() throws org.apache.thrift.TException - { - GetOperationStatus_result result = new GetOperationStatus_result(); - receiveBase(result, "GetOperationStatus"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetOperationStatus failed: unknown result"); - } - - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException - { - send_CancelOperation(req); - return recv_CancelOperation(); - } - - public void send_CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException - { - CancelOperation_args args = new CancelOperation_args(); - args.setReq(req); - sendBase("CancelOperation", args); - } - - public TCancelOperationResp recv_CancelOperation() throws org.apache.thrift.TException - { - CancelOperation_result result = new CancelOperation_result(); - receiveBase(result, "CancelOperation"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelOperation failed: unknown result"); - } - - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException - { - send_CloseOperation(req); - return recv_CloseOperation(); - } - - public void send_CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException - { - CloseOperation_args args = new CloseOperation_args(); - args.setReq(req); - sendBase("CloseOperation", args); - } - - public TCloseOperationResp recv_CloseOperation() throws org.apache.thrift.TException - { - CloseOperation_result result = new CloseOperation_result(); - receiveBase(result, "CloseOperation"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseOperation failed: unknown result"); - } - - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException - { - send_GetResultSetMetadata(req); - return recv_GetResultSetMetadata(); - } - - public void send_GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException - { - GetResultSetMetadata_args args = new GetResultSetMetadata_args(); - args.setReq(req); - sendBase("GetResultSetMetadata", args); - } - - public TGetResultSetMetadataResp recv_GetResultSetMetadata() throws org.apache.thrift.TException - { - GetResultSetMetadata_result result = new GetResultSetMetadata_result(); - receiveBase(result, "GetResultSetMetadata"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetResultSetMetadata failed: unknown result"); - } - - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException - { - send_FetchResults(req); - return recv_FetchResults(); - } - - public void send_FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException - { - FetchResults_args args = new FetchResults_args(); - args.setReq(req); - sendBase("FetchResults", args); - } - - public TFetchResultsResp recv_FetchResults() throws org.apache.thrift.TException - { - FetchResults_result result = new FetchResults_result(); - receiveBase(result, "FetchResults"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "FetchResults failed: unknown result"); - } - - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException - { - send_GetDelegationToken(req); - return recv_GetDelegationToken(); - } - - public void send_GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException - { - GetDelegationToken_args args = new GetDelegationToken_args(); - args.setReq(req); - sendBase("GetDelegationToken", args); - } - - public TGetDelegationTokenResp recv_GetDelegationToken() throws org.apache.thrift.TException - { - GetDelegationToken_result result = new GetDelegationToken_result(); - receiveBase(result, "GetDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetDelegationToken failed: unknown result"); - } - - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException - { - send_CancelDelegationToken(req); - return recv_CancelDelegationToken(); - } - - public void send_CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException - { - CancelDelegationToken_args args = new CancelDelegationToken_args(); - args.setReq(req); - sendBase("CancelDelegationToken", args); - } - - public TCancelDelegationTokenResp recv_CancelDelegationToken() throws org.apache.thrift.TException - { - CancelDelegationToken_result result = new CancelDelegationToken_result(); - receiveBase(result, "CancelDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelDelegationToken failed: unknown result"); - } - - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException - { - send_RenewDelegationToken(req); - return recv_RenewDelegationToken(); - } - - public void send_RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException - { - RenewDelegationToken_args args = new RenewDelegationToken_args(); - args.setReq(req); - sendBase("RenewDelegationToken", args); - } - - public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.thrift.TException - { - RenewDelegationToken_result result = new RenewDelegationToken_result(); - receiveBase(result, "RenewDelegationToken"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result"); - } - - } - public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { - public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { - private org.apache.thrift.async.TAsyncClientManager clientManager; - private org.apache.thrift.protocol.TProtocolFactory protocolFactory; - public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { - this.clientManager = clientManager; - this.protocolFactory = protocolFactory; - } - public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { - return new AsyncClient(protocolFactory, clientManager, transport); - } - } - - public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { - super(protocolFactory, clientManager, transport); - } - - public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - OpenSession_call method_call = new OpenSession_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class OpenSession_call extends org.apache.thrift.async.TAsyncMethodCall { - private TOpenSessionReq req; - public OpenSession_call(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("OpenSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); - OpenSession_args args = new OpenSession_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TOpenSessionResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_OpenSession(); - } - } - - public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CloseSession_call method_call = new CloseSession_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CloseSession_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCloseSessionReq req; - public CloseSession_call(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CloseSession_args args = new CloseSession_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCloseSessionResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CloseSession(); - } - } - - public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetInfo_call method_call = new GetInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetInfo_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetInfoReq req; - public GetInfo_call(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetInfo_args args = new GetInfo_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetInfoResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetInfo(); - } - } - - public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - ExecuteStatement_call method_call = new ExecuteStatement_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class ExecuteStatement_call extends org.apache.thrift.async.TAsyncMethodCall { - private TExecuteStatementReq req; - public ExecuteStatement_call(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("ExecuteStatement", org.apache.thrift.protocol.TMessageType.CALL, 0)); - ExecuteStatement_args args = new ExecuteStatement_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TExecuteStatementResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_ExecuteStatement(); - } - } - - public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTypeInfo_call method_call = new GetTypeInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTypeInfo_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTypeInfoReq req; - public GetTypeInfo_call(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTypeInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTypeInfo_args args = new GetTypeInfo_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTypeInfoResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTypeInfo(); - } - } - - public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetCatalogs_call method_call = new GetCatalogs_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetCatalogs_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetCatalogsReq req; - public GetCatalogs_call(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCatalogs", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetCatalogs_args args = new GetCatalogs_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetCatalogsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetCatalogs(); - } - } - - public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetSchemas_call method_call = new GetSchemas_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetSchemas_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetSchemasReq req; - public GetSchemas_call(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetSchemas", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetSchemas_args args = new GetSchemas_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetSchemasResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetSchemas(); - } - } - - public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTables_call method_call = new GetTables_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTables_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTablesReq req; - public GetTables_call(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTables", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTables_args args = new GetTables_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTablesResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTables(); - } - } - - public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetTableTypes_call method_call = new GetTableTypes_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetTableTypes_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetTableTypesReq req; - public GetTableTypes_call(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTableTypes", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetTableTypes_args args = new GetTableTypes_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetTableTypesResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetTableTypes(); - } - } - - public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetColumns_call method_call = new GetColumns_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetColumns_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetColumnsReq req; - public GetColumns_call(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetColumns", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetColumns_args args = new GetColumns_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetColumnsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetColumns(); - } - } - - public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetFunctions_call method_call = new GetFunctions_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetFunctions_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetFunctionsReq req; - public GetFunctions_call(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetFunctions", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetFunctions_args args = new GetFunctions_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetFunctionsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetFunctions(); - } - } - - public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetOperationStatus_call method_call = new GetOperationStatus_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetOperationStatus_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetOperationStatusReq req; - public GetOperationStatus_call(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetOperationStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetOperationStatus_args args = new GetOperationStatus_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetOperationStatusResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetOperationStatus(); - } - } - - public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CancelOperation_call method_call = new CancelOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CancelOperation_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCancelOperationReq req; - public CancelOperation_call(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CancelOperation_args args = new CancelOperation_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCancelOperationResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CancelOperation(); - } - } - - public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CloseOperation_call method_call = new CloseOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CloseOperation_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCloseOperationReq req; - public CloseOperation_call(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CloseOperation_args args = new CloseOperation_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCloseOperationResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CloseOperation(); - } - } - - public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetResultSetMetadata_call method_call = new GetResultSetMetadata_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetResultSetMetadata_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetResultSetMetadataReq req; - public GetResultSetMetadata_call(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetResultSetMetadata", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetResultSetMetadata_args args = new GetResultSetMetadata_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetResultSetMetadataResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetResultSetMetadata(); - } - } - - public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - FetchResults_call method_call = new FetchResults_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class FetchResults_call extends org.apache.thrift.async.TAsyncMethodCall { - private TFetchResultsReq req; - public FetchResults_call(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("FetchResults", org.apache.thrift.protocol.TMessageType.CALL, 0)); - FetchResults_args args = new FetchResults_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TFetchResultsResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_FetchResults(); - } - } - - public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - GetDelegationToken_call method_call = new GetDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class GetDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TGetDelegationTokenReq req; - public GetDelegationToken_call(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - GetDelegationToken_args args = new GetDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TGetDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_GetDelegationToken(); - } - } - - public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - CancelDelegationToken_call method_call = new CancelDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class CancelDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TCancelDelegationTokenReq req; - public CancelDelegationToken_call(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - CancelDelegationToken_args args = new CancelDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TCancelDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_CancelDelegationToken(); - } - } - - public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - RenewDelegationToken_call method_call = new RenewDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class RenewDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { - private TRenewDelegationTokenReq req; - public RenewDelegationToken_call(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.req = req; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("RenewDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); - RenewDelegationToken_args args = new RenewDelegationToken_args(); - args.setReq(req); - args.write(prot); - prot.writeMessageEnd(); - } - - public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_RenewDelegationToken(); - } - } - - } - - public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); - public Processor(I iface) { - super(iface, getProcessMap(new HashMap>())); - } - - protected Processor(I iface, Map> processMap) { - super(iface, getProcessMap(processMap)); - } - - private static Map> getProcessMap(Map> processMap) { - processMap.put("OpenSession", new OpenSession()); - processMap.put("CloseSession", new CloseSession()); - processMap.put("GetInfo", new GetInfo()); - processMap.put("ExecuteStatement", new ExecuteStatement()); - processMap.put("GetTypeInfo", new GetTypeInfo()); - processMap.put("GetCatalogs", new GetCatalogs()); - processMap.put("GetSchemas", new GetSchemas()); - processMap.put("GetTables", new GetTables()); - processMap.put("GetTableTypes", new GetTableTypes()); - processMap.put("GetColumns", new GetColumns()); - processMap.put("GetFunctions", new GetFunctions()); - processMap.put("GetOperationStatus", new GetOperationStatus()); - processMap.put("CancelOperation", new CancelOperation()); - processMap.put("CloseOperation", new CloseOperation()); - processMap.put("GetResultSetMetadata", new GetResultSetMetadata()); - processMap.put("FetchResults", new FetchResults()); - processMap.put("GetDelegationToken", new GetDelegationToken()); - processMap.put("CancelDelegationToken", new CancelDelegationToken()); - processMap.put("RenewDelegationToken", new RenewDelegationToken()); - return processMap; - } - - public static class OpenSession extends org.apache.thrift.ProcessFunction { - public OpenSession() { - super("OpenSession"); - } - - public OpenSession_args getEmptyArgsInstance() { - return new OpenSession_args(); - } - - protected boolean isOneway() { - return false; - } - - public OpenSession_result getResult(I iface, OpenSession_args args) throws org.apache.thrift.TException { - OpenSession_result result = new OpenSession_result(); - result.success = iface.OpenSession(args.req); - return result; - } - } - - public static class CloseSession extends org.apache.thrift.ProcessFunction { - public CloseSession() { - super("CloseSession"); - } - - public CloseSession_args getEmptyArgsInstance() { - return new CloseSession_args(); - } - - protected boolean isOneway() { - return false; - } - - public CloseSession_result getResult(I iface, CloseSession_args args) throws org.apache.thrift.TException { - CloseSession_result result = new CloseSession_result(); - result.success = iface.CloseSession(args.req); - return result; - } - } - - public static class GetInfo extends org.apache.thrift.ProcessFunction { - public GetInfo() { - super("GetInfo"); - } - - public GetInfo_args getEmptyArgsInstance() { - return new GetInfo_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetInfo_result getResult(I iface, GetInfo_args args) throws org.apache.thrift.TException { - GetInfo_result result = new GetInfo_result(); - result.success = iface.GetInfo(args.req); - return result; - } - } - - public static class ExecuteStatement extends org.apache.thrift.ProcessFunction { - public ExecuteStatement() { - super("ExecuteStatement"); - } - - public ExecuteStatement_args getEmptyArgsInstance() { - return new ExecuteStatement_args(); - } - - protected boolean isOneway() { - return false; - } - - public ExecuteStatement_result getResult(I iface, ExecuteStatement_args args) throws org.apache.thrift.TException { - ExecuteStatement_result result = new ExecuteStatement_result(); - result.success = iface.ExecuteStatement(args.req); - return result; - } - } - - public static class GetTypeInfo extends org.apache.thrift.ProcessFunction { - public GetTypeInfo() { - super("GetTypeInfo"); - } - - public GetTypeInfo_args getEmptyArgsInstance() { - return new GetTypeInfo_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTypeInfo_result getResult(I iface, GetTypeInfo_args args) throws org.apache.thrift.TException { - GetTypeInfo_result result = new GetTypeInfo_result(); - result.success = iface.GetTypeInfo(args.req); - return result; - } - } - - public static class GetCatalogs extends org.apache.thrift.ProcessFunction { - public GetCatalogs() { - super("GetCatalogs"); - } - - public GetCatalogs_args getEmptyArgsInstance() { - return new GetCatalogs_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetCatalogs_result getResult(I iface, GetCatalogs_args args) throws org.apache.thrift.TException { - GetCatalogs_result result = new GetCatalogs_result(); - result.success = iface.GetCatalogs(args.req); - return result; - } - } - - public static class GetSchemas extends org.apache.thrift.ProcessFunction { - public GetSchemas() { - super("GetSchemas"); - } - - public GetSchemas_args getEmptyArgsInstance() { - return new GetSchemas_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetSchemas_result getResult(I iface, GetSchemas_args args) throws org.apache.thrift.TException { - GetSchemas_result result = new GetSchemas_result(); - result.success = iface.GetSchemas(args.req); - return result; - } - } - - public static class GetTables extends org.apache.thrift.ProcessFunction { - public GetTables() { - super("GetTables"); - } - - public GetTables_args getEmptyArgsInstance() { - return new GetTables_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTables_result getResult(I iface, GetTables_args args) throws org.apache.thrift.TException { - GetTables_result result = new GetTables_result(); - result.success = iface.GetTables(args.req); - return result; - } - } - - public static class GetTableTypes extends org.apache.thrift.ProcessFunction { - public GetTableTypes() { - super("GetTableTypes"); - } - - public GetTableTypes_args getEmptyArgsInstance() { - return new GetTableTypes_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetTableTypes_result getResult(I iface, GetTableTypes_args args) throws org.apache.thrift.TException { - GetTableTypes_result result = new GetTableTypes_result(); - result.success = iface.GetTableTypes(args.req); - return result; - } - } - - public static class GetColumns extends org.apache.thrift.ProcessFunction { - public GetColumns() { - super("GetColumns"); - } - - public GetColumns_args getEmptyArgsInstance() { - return new GetColumns_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetColumns_result getResult(I iface, GetColumns_args args) throws org.apache.thrift.TException { - GetColumns_result result = new GetColumns_result(); - result.success = iface.GetColumns(args.req); - return result; - } - } - - public static class GetFunctions extends org.apache.thrift.ProcessFunction { - public GetFunctions() { - super("GetFunctions"); - } - - public GetFunctions_args getEmptyArgsInstance() { - return new GetFunctions_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetFunctions_result getResult(I iface, GetFunctions_args args) throws org.apache.thrift.TException { - GetFunctions_result result = new GetFunctions_result(); - result.success = iface.GetFunctions(args.req); - return result; - } - } - - public static class GetOperationStatus extends org.apache.thrift.ProcessFunction { - public GetOperationStatus() { - super("GetOperationStatus"); - } - - public GetOperationStatus_args getEmptyArgsInstance() { - return new GetOperationStatus_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetOperationStatus_result getResult(I iface, GetOperationStatus_args args) throws org.apache.thrift.TException { - GetOperationStatus_result result = new GetOperationStatus_result(); - result.success = iface.GetOperationStatus(args.req); - return result; - } - } - - public static class CancelOperation extends org.apache.thrift.ProcessFunction { - public CancelOperation() { - super("CancelOperation"); - } - - public CancelOperation_args getEmptyArgsInstance() { - return new CancelOperation_args(); - } - - protected boolean isOneway() { - return false; - } - - public CancelOperation_result getResult(I iface, CancelOperation_args args) throws org.apache.thrift.TException { - CancelOperation_result result = new CancelOperation_result(); - result.success = iface.CancelOperation(args.req); - return result; - } - } - - public static class CloseOperation extends org.apache.thrift.ProcessFunction { - public CloseOperation() { - super("CloseOperation"); - } - - public CloseOperation_args getEmptyArgsInstance() { - return new CloseOperation_args(); - } - - protected boolean isOneway() { - return false; - } - - public CloseOperation_result getResult(I iface, CloseOperation_args args) throws org.apache.thrift.TException { - CloseOperation_result result = new CloseOperation_result(); - result.success = iface.CloseOperation(args.req); - return result; - } - } - - public static class GetResultSetMetadata extends org.apache.thrift.ProcessFunction { - public GetResultSetMetadata() { - super("GetResultSetMetadata"); - } - - public GetResultSetMetadata_args getEmptyArgsInstance() { - return new GetResultSetMetadata_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetResultSetMetadata_result getResult(I iface, GetResultSetMetadata_args args) throws org.apache.thrift.TException { - GetResultSetMetadata_result result = new GetResultSetMetadata_result(); - result.success = iface.GetResultSetMetadata(args.req); - return result; - } - } - - public static class FetchResults extends org.apache.thrift.ProcessFunction { - public FetchResults() { - super("FetchResults"); - } - - public FetchResults_args getEmptyArgsInstance() { - return new FetchResults_args(); - } - - protected boolean isOneway() { - return false; - } - - public FetchResults_result getResult(I iface, FetchResults_args args) throws org.apache.thrift.TException { - FetchResults_result result = new FetchResults_result(); - result.success = iface.FetchResults(args.req); - return result; - } - } - - public static class GetDelegationToken extends org.apache.thrift.ProcessFunction { - public GetDelegationToken() { - super("GetDelegationToken"); - } - - public GetDelegationToken_args getEmptyArgsInstance() { - return new GetDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public GetDelegationToken_result getResult(I iface, GetDelegationToken_args args) throws org.apache.thrift.TException { - GetDelegationToken_result result = new GetDelegationToken_result(); - result.success = iface.GetDelegationToken(args.req); - return result; - } - } - - public static class CancelDelegationToken extends org.apache.thrift.ProcessFunction { - public CancelDelegationToken() { - super("CancelDelegationToken"); - } - - public CancelDelegationToken_args getEmptyArgsInstance() { - return new CancelDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public CancelDelegationToken_result getResult(I iface, CancelDelegationToken_args args) throws org.apache.thrift.TException { - CancelDelegationToken_result result = new CancelDelegationToken_result(); - result.success = iface.CancelDelegationToken(args.req); - return result; - } - } - - public static class RenewDelegationToken extends org.apache.thrift.ProcessFunction { - public RenewDelegationToken() { - super("RenewDelegationToken"); - } - - public RenewDelegationToken_args getEmptyArgsInstance() { - return new RenewDelegationToken_args(); - } - - protected boolean isOneway() { - return false; - } - - public RenewDelegationToken_result getResult(I iface, RenewDelegationToken_args args) throws org.apache.thrift.TException { - RenewDelegationToken_result result = new RenewDelegationToken_result(); - result.success = iface.RenewDelegationToken(args.req); - return result; - } - } - - } - - public static class OpenSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new OpenSession_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new OpenSession_argsTupleSchemeFactory()); - } - - private TOpenSessionReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_args.class, metaDataMap); - } - - public OpenSession_args() { - } - - public OpenSession_args( - TOpenSessionReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public OpenSession_args(OpenSession_args other) { - if (other.isSetReq()) { - this.req = new TOpenSessionReq(other.req); - } - } - - public OpenSession_args deepCopy() { - return new OpenSession_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TOpenSessionReq getReq() { - return this.req; - } - - public void setReq(TOpenSessionReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TOpenSessionReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof OpenSession_args) - return this.equals((OpenSession_args)that); - return false; - } - - public boolean equals(OpenSession_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(OpenSession_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - OpenSession_args typedOther = (OpenSession_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("OpenSession_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class OpenSession_argsStandardSchemeFactory implements SchemeFactory { - public OpenSession_argsStandardScheme getScheme() { - return new OpenSession_argsStandardScheme(); - } - } - - private static class OpenSession_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TOpenSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class OpenSession_argsTupleSchemeFactory implements SchemeFactory { - public OpenSession_argsTupleScheme getScheme() { - return new OpenSession_argsTupleScheme(); - } - } - - private static class OpenSession_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TOpenSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class OpenSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new OpenSession_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new OpenSession_resultTupleSchemeFactory()); - } - - private TOpenSessionResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_result.class, metaDataMap); - } - - public OpenSession_result() { - } - - public OpenSession_result( - TOpenSessionResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public OpenSession_result(OpenSession_result other) { - if (other.isSetSuccess()) { - this.success = new TOpenSessionResp(other.success); - } - } - - public OpenSession_result deepCopy() { - return new OpenSession_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TOpenSessionResp getSuccess() { - return this.success; - } - - public void setSuccess(TOpenSessionResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TOpenSessionResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof OpenSession_result) - return this.equals((OpenSession_result)that); - return false; - } - - public boolean equals(OpenSession_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(OpenSession_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - OpenSession_result typedOther = (OpenSession_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("OpenSession_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class OpenSession_resultStandardSchemeFactory implements SchemeFactory { - public OpenSession_resultStandardScheme getScheme() { - return new OpenSession_resultStandardScheme(); - } - } - - private static class OpenSession_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TOpenSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class OpenSession_resultTupleSchemeFactory implements SchemeFactory { - public OpenSession_resultTupleScheme getScheme() { - return new OpenSession_resultTupleScheme(); - } - } - - private static class OpenSession_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TOpenSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CloseSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseSession_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseSession_argsTupleSchemeFactory()); - } - - private TCloseSessionReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_args.class, metaDataMap); - } - - public CloseSession_args() { - } - - public CloseSession_args( - TCloseSessionReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CloseSession_args(CloseSession_args other) { - if (other.isSetReq()) { - this.req = new TCloseSessionReq(other.req); - } - } - - public CloseSession_args deepCopy() { - return new CloseSession_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCloseSessionReq getReq() { - return this.req; - } - - public void setReq(TCloseSessionReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCloseSessionReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseSession_args) - return this.equals((CloseSession_args)that); - return false; - } - - public boolean equals(CloseSession_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CloseSession_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseSession_args typedOther = (CloseSession_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseSession_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseSession_argsStandardSchemeFactory implements SchemeFactory { - public CloseSession_argsStandardScheme getScheme() { - return new CloseSession_argsStandardScheme(); - } - } - - private static class CloseSession_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCloseSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseSession_argsTupleSchemeFactory implements SchemeFactory { - public CloseSession_argsTupleScheme getScheme() { - return new CloseSession_argsTupleScheme(); - } - } - - private static class CloseSession_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCloseSessionReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CloseSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseSession_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseSession_resultTupleSchemeFactory()); - } - - private TCloseSessionResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_result.class, metaDataMap); - } - - public CloseSession_result() { - } - - public CloseSession_result( - TCloseSessionResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CloseSession_result(CloseSession_result other) { - if (other.isSetSuccess()) { - this.success = new TCloseSessionResp(other.success); - } - } - - public CloseSession_result deepCopy() { - return new CloseSession_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCloseSessionResp getSuccess() { - return this.success; - } - - public void setSuccess(TCloseSessionResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCloseSessionResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseSession_result) - return this.equals((CloseSession_result)that); - return false; - } - - public boolean equals(CloseSession_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CloseSession_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseSession_result typedOther = (CloseSession_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseSession_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseSession_resultStandardSchemeFactory implements SchemeFactory { - public CloseSession_resultStandardScheme getScheme() { - return new CloseSession_resultStandardScheme(); - } - } - - private static class CloseSession_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCloseSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseSession_resultTupleSchemeFactory implements SchemeFactory { - public CloseSession_resultTupleScheme getScheme() { - return new CloseSession_resultTupleScheme(); - } - } - - private static class CloseSession_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCloseSessionResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetInfo_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetInfo_argsTupleSchemeFactory()); - } - - private TGetInfoReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_args.class, metaDataMap); - } - - public GetInfo_args() { - } - - public GetInfo_args( - TGetInfoReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetInfo_args(GetInfo_args other) { - if (other.isSetReq()) { - this.req = new TGetInfoReq(other.req); - } - } - - public GetInfo_args deepCopy() { - return new GetInfo_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetInfoReq getReq() { - return this.req; - } - - public void setReq(TGetInfoReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetInfoReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetInfo_args) - return this.equals((GetInfo_args)that); - return false; - } - - public boolean equals(GetInfo_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetInfo_args typedOther = (GetInfo_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetInfo_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetInfo_argsStandardSchemeFactory implements SchemeFactory { - public GetInfo_argsStandardScheme getScheme() { - return new GetInfo_argsStandardScheme(); - } - } - - private static class GetInfo_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetInfo_argsTupleSchemeFactory implements SchemeFactory { - public GetInfo_argsTupleScheme getScheme() { - return new GetInfo_argsTupleScheme(); - } - } - - private static class GetInfo_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetInfo_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetInfo_resultTupleSchemeFactory()); - } - - private TGetInfoResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_result.class, metaDataMap); - } - - public GetInfo_result() { - } - - public GetInfo_result( - TGetInfoResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetInfo_result(GetInfo_result other) { - if (other.isSetSuccess()) { - this.success = new TGetInfoResp(other.success); - } - } - - public GetInfo_result deepCopy() { - return new GetInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetInfoResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetInfoResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetInfoResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetInfo_result) - return this.equals((GetInfo_result)that); - return false; - } - - public boolean equals(GetInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetInfo_result typedOther = (GetInfo_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetInfo_resultStandardSchemeFactory implements SchemeFactory { - public GetInfo_resultStandardScheme getScheme() { - return new GetInfo_resultStandardScheme(); - } - } - - private static class GetInfo_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetInfo_resultTupleSchemeFactory implements SchemeFactory { - public GetInfo_resultTupleScheme getScheme() { - return new GetInfo_resultTupleScheme(); - } - } - - private static class GetInfo_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class ExecuteStatement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new ExecuteStatement_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new ExecuteStatement_argsTupleSchemeFactory()); - } - - private TExecuteStatementReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_args.class, metaDataMap); - } - - public ExecuteStatement_args() { - } - - public ExecuteStatement_args( - TExecuteStatementReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public ExecuteStatement_args(ExecuteStatement_args other) { - if (other.isSetReq()) { - this.req = new TExecuteStatementReq(other.req); - } - } - - public ExecuteStatement_args deepCopy() { - return new ExecuteStatement_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TExecuteStatementReq getReq() { - return this.req; - } - - public void setReq(TExecuteStatementReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TExecuteStatementReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ExecuteStatement_args) - return this.equals((ExecuteStatement_args)that); - return false; - } - - public boolean equals(ExecuteStatement_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(ExecuteStatement_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ExecuteStatement_args typedOther = (ExecuteStatement_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ExecuteStatement_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class ExecuteStatement_argsStandardSchemeFactory implements SchemeFactory { - public ExecuteStatement_argsStandardScheme getScheme() { - return new ExecuteStatement_argsStandardScheme(); - } - } - - private static class ExecuteStatement_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TExecuteStatementReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class ExecuteStatement_argsTupleSchemeFactory implements SchemeFactory { - public ExecuteStatement_argsTupleScheme getScheme() { - return new ExecuteStatement_argsTupleScheme(); - } - } - - private static class ExecuteStatement_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TExecuteStatementReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class ExecuteStatement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new ExecuteStatement_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new ExecuteStatement_resultTupleSchemeFactory()); - } - - private TExecuteStatementResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_result.class, metaDataMap); - } - - public ExecuteStatement_result() { - } - - public ExecuteStatement_result( - TExecuteStatementResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public ExecuteStatement_result(ExecuteStatement_result other) { - if (other.isSetSuccess()) { - this.success = new TExecuteStatementResp(other.success); - } - } - - public ExecuteStatement_result deepCopy() { - return new ExecuteStatement_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TExecuteStatementResp getSuccess() { - return this.success; - } - - public void setSuccess(TExecuteStatementResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TExecuteStatementResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ExecuteStatement_result) - return this.equals((ExecuteStatement_result)that); - return false; - } - - public boolean equals(ExecuteStatement_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(ExecuteStatement_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ExecuteStatement_result typedOther = (ExecuteStatement_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ExecuteStatement_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class ExecuteStatement_resultStandardSchemeFactory implements SchemeFactory { - public ExecuteStatement_resultStandardScheme getScheme() { - return new ExecuteStatement_resultStandardScheme(); - } - } - - private static class ExecuteStatement_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TExecuteStatementResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class ExecuteStatement_resultTupleSchemeFactory implements SchemeFactory { - public ExecuteStatement_resultTupleScheme getScheme() { - return new ExecuteStatement_resultTupleScheme(); - } - } - - private static class ExecuteStatement_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TExecuteStatementResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTypeInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTypeInfo_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTypeInfo_argsTupleSchemeFactory()); - } - - private TGetTypeInfoReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_args.class, metaDataMap); - } - - public GetTypeInfo_args() { - } - - public GetTypeInfo_args( - TGetTypeInfoReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTypeInfo_args(GetTypeInfo_args other) { - if (other.isSetReq()) { - this.req = new TGetTypeInfoReq(other.req); - } - } - - public GetTypeInfo_args deepCopy() { - return new GetTypeInfo_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTypeInfoReq getReq() { - return this.req; - } - - public void setReq(TGetTypeInfoReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTypeInfoReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTypeInfo_args) - return this.equals((GetTypeInfo_args)that); - return false; - } - - public boolean equals(GetTypeInfo_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTypeInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTypeInfo_args typedOther = (GetTypeInfo_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTypeInfo_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTypeInfo_argsStandardSchemeFactory implements SchemeFactory { - public GetTypeInfo_argsStandardScheme getScheme() { - return new GetTypeInfo_argsStandardScheme(); - } - } - - private static class GetTypeInfo_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTypeInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTypeInfo_argsTupleSchemeFactory implements SchemeFactory { - public GetTypeInfo_argsTupleScheme getScheme() { - return new GetTypeInfo_argsTupleScheme(); - } - } - - private static class GetTypeInfo_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTypeInfoReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTypeInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTypeInfo_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTypeInfo_resultTupleSchemeFactory()); - } - - private TGetTypeInfoResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_result.class, metaDataMap); - } - - public GetTypeInfo_result() { - } - - public GetTypeInfo_result( - TGetTypeInfoResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTypeInfo_result(GetTypeInfo_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTypeInfoResp(other.success); - } - } - - public GetTypeInfo_result deepCopy() { - return new GetTypeInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTypeInfoResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTypeInfoResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTypeInfoResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTypeInfo_result) - return this.equals((GetTypeInfo_result)that); - return false; - } - - public boolean equals(GetTypeInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTypeInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTypeInfo_result typedOther = (GetTypeInfo_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTypeInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTypeInfo_resultStandardSchemeFactory implements SchemeFactory { - public GetTypeInfo_resultStandardScheme getScheme() { - return new GetTypeInfo_resultStandardScheme(); - } - } - - private static class GetTypeInfo_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTypeInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTypeInfo_resultTupleSchemeFactory implements SchemeFactory { - public GetTypeInfo_resultTupleScheme getScheme() { - return new GetTypeInfo_resultTupleScheme(); - } - } - - private static class GetTypeInfo_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTypeInfoResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetCatalogs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetCatalogs_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetCatalogs_argsTupleSchemeFactory()); - } - - private TGetCatalogsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_args.class, metaDataMap); - } - - public GetCatalogs_args() { - } - - public GetCatalogs_args( - TGetCatalogsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetCatalogs_args(GetCatalogs_args other) { - if (other.isSetReq()) { - this.req = new TGetCatalogsReq(other.req); - } - } - - public GetCatalogs_args deepCopy() { - return new GetCatalogs_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetCatalogsReq getReq() { - return this.req; - } - - public void setReq(TGetCatalogsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetCatalogsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetCatalogs_args) - return this.equals((GetCatalogs_args)that); - return false; - } - - public boolean equals(GetCatalogs_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetCatalogs_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetCatalogs_args typedOther = (GetCatalogs_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetCatalogs_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetCatalogs_argsStandardSchemeFactory implements SchemeFactory { - public GetCatalogs_argsStandardScheme getScheme() { - return new GetCatalogs_argsStandardScheme(); - } - } - - private static class GetCatalogs_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetCatalogsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetCatalogs_argsTupleSchemeFactory implements SchemeFactory { - public GetCatalogs_argsTupleScheme getScheme() { - return new GetCatalogs_argsTupleScheme(); - } - } - - private static class GetCatalogs_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetCatalogsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetCatalogs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetCatalogs_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetCatalogs_resultTupleSchemeFactory()); - } - - private TGetCatalogsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_result.class, metaDataMap); - } - - public GetCatalogs_result() { - } - - public GetCatalogs_result( - TGetCatalogsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetCatalogs_result(GetCatalogs_result other) { - if (other.isSetSuccess()) { - this.success = new TGetCatalogsResp(other.success); - } - } - - public GetCatalogs_result deepCopy() { - return new GetCatalogs_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetCatalogsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetCatalogsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetCatalogsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetCatalogs_result) - return this.equals((GetCatalogs_result)that); - return false; - } - - public boolean equals(GetCatalogs_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetCatalogs_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetCatalogs_result typedOther = (GetCatalogs_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetCatalogs_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetCatalogs_resultStandardSchemeFactory implements SchemeFactory { - public GetCatalogs_resultStandardScheme getScheme() { - return new GetCatalogs_resultStandardScheme(); - } - } - - private static class GetCatalogs_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetCatalogsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetCatalogs_resultTupleSchemeFactory implements SchemeFactory { - public GetCatalogs_resultTupleScheme getScheme() { - return new GetCatalogs_resultTupleScheme(); - } - } - - private static class GetCatalogs_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetCatalogsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetSchemas_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetSchemas_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetSchemas_argsTupleSchemeFactory()); - } - - private TGetSchemasReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_args.class, metaDataMap); - } - - public GetSchemas_args() { - } - - public GetSchemas_args( - TGetSchemasReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetSchemas_args(GetSchemas_args other) { - if (other.isSetReq()) { - this.req = new TGetSchemasReq(other.req); - } - } - - public GetSchemas_args deepCopy() { - return new GetSchemas_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetSchemasReq getReq() { - return this.req; - } - - public void setReq(TGetSchemasReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetSchemasReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetSchemas_args) - return this.equals((GetSchemas_args)that); - return false; - } - - public boolean equals(GetSchemas_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetSchemas_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetSchemas_args typedOther = (GetSchemas_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetSchemas_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetSchemas_argsStandardSchemeFactory implements SchemeFactory { - public GetSchemas_argsStandardScheme getScheme() { - return new GetSchemas_argsStandardScheme(); - } - } - - private static class GetSchemas_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetSchemasReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetSchemas_argsTupleSchemeFactory implements SchemeFactory { - public GetSchemas_argsTupleScheme getScheme() { - return new GetSchemas_argsTupleScheme(); - } - } - - private static class GetSchemas_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetSchemasReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetSchemas_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetSchemas_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetSchemas_resultTupleSchemeFactory()); - } - - private TGetSchemasResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_result.class, metaDataMap); - } - - public GetSchemas_result() { - } - - public GetSchemas_result( - TGetSchemasResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetSchemas_result(GetSchemas_result other) { - if (other.isSetSuccess()) { - this.success = new TGetSchemasResp(other.success); - } - } - - public GetSchemas_result deepCopy() { - return new GetSchemas_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetSchemasResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetSchemasResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetSchemasResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetSchemas_result) - return this.equals((GetSchemas_result)that); - return false; - } - - public boolean equals(GetSchemas_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetSchemas_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetSchemas_result typedOther = (GetSchemas_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetSchemas_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetSchemas_resultStandardSchemeFactory implements SchemeFactory { - public GetSchemas_resultStandardScheme getScheme() { - return new GetSchemas_resultStandardScheme(); - } - } - - private static class GetSchemas_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetSchemasResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetSchemas_resultTupleSchemeFactory implements SchemeFactory { - public GetSchemas_resultTupleScheme getScheme() { - return new GetSchemas_resultTupleScheme(); - } - } - - private static class GetSchemas_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetSchemasResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTables_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTables_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTables_argsTupleSchemeFactory()); - } - - private TGetTablesReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_args.class, metaDataMap); - } - - public GetTables_args() { - } - - public GetTables_args( - TGetTablesReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTables_args(GetTables_args other) { - if (other.isSetReq()) { - this.req = new TGetTablesReq(other.req); - } - } - - public GetTables_args deepCopy() { - return new GetTables_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTablesReq getReq() { - return this.req; - } - - public void setReq(TGetTablesReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTablesReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTables_args) - return this.equals((GetTables_args)that); - return false; - } - - public boolean equals(GetTables_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTables_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTables_args typedOther = (GetTables_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTables_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTables_argsStandardSchemeFactory implements SchemeFactory { - public GetTables_argsStandardScheme getScheme() { - return new GetTables_argsStandardScheme(); - } - } - - private static class GetTables_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTablesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTables_argsTupleSchemeFactory implements SchemeFactory { - public GetTables_argsTupleScheme getScheme() { - return new GetTables_argsTupleScheme(); - } - } - - private static class GetTables_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTablesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTables_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTables_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTables_resultTupleSchemeFactory()); - } - - private TGetTablesResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_result.class, metaDataMap); - } - - public GetTables_result() { - } - - public GetTables_result( - TGetTablesResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTables_result(GetTables_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTablesResp(other.success); - } - } - - public GetTables_result deepCopy() { - return new GetTables_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTablesResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTablesResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTablesResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTables_result) - return this.equals((GetTables_result)that); - return false; - } - - public boolean equals(GetTables_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTables_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTables_result typedOther = (GetTables_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTables_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTables_resultStandardSchemeFactory implements SchemeFactory { - public GetTables_resultStandardScheme getScheme() { - return new GetTables_resultStandardScheme(); - } - } - - private static class GetTables_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTablesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTables_resultTupleSchemeFactory implements SchemeFactory { - public GetTables_resultTupleScheme getScheme() { - return new GetTables_resultTupleScheme(); - } - } - - private static class GetTables_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTablesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetTableTypes_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTableTypes_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTableTypes_argsTupleSchemeFactory()); - } - - private TGetTableTypesReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_args.class, metaDataMap); - } - - public GetTableTypes_args() { - } - - public GetTableTypes_args( - TGetTableTypesReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetTableTypes_args(GetTableTypes_args other) { - if (other.isSetReq()) { - this.req = new TGetTableTypesReq(other.req); - } - } - - public GetTableTypes_args deepCopy() { - return new GetTableTypes_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetTableTypesReq getReq() { - return this.req; - } - - public void setReq(TGetTableTypesReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetTableTypesReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTableTypes_args) - return this.equals((GetTableTypes_args)that); - return false; - } - - public boolean equals(GetTableTypes_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetTableTypes_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTableTypes_args typedOther = (GetTableTypes_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTableTypes_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTableTypes_argsStandardSchemeFactory implements SchemeFactory { - public GetTableTypes_argsStandardScheme getScheme() { - return new GetTableTypes_argsStandardScheme(); - } - } - - private static class GetTableTypes_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetTableTypesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTableTypes_argsTupleSchemeFactory implements SchemeFactory { - public GetTableTypes_argsTupleScheme getScheme() { - return new GetTableTypes_argsTupleScheme(); - } - } - - private static class GetTableTypes_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetTableTypesReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetTableTypes_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetTableTypes_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetTableTypes_resultTupleSchemeFactory()); - } - - private TGetTableTypesResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_result.class, metaDataMap); - } - - public GetTableTypes_result() { - } - - public GetTableTypes_result( - TGetTableTypesResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetTableTypes_result(GetTableTypes_result other) { - if (other.isSetSuccess()) { - this.success = new TGetTableTypesResp(other.success); - } - } - - public GetTableTypes_result deepCopy() { - return new GetTableTypes_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetTableTypesResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetTableTypesResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetTableTypesResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetTableTypes_result) - return this.equals((GetTableTypes_result)that); - return false; - } - - public boolean equals(GetTableTypes_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetTableTypes_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetTableTypes_result typedOther = (GetTableTypes_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetTableTypes_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetTableTypes_resultStandardSchemeFactory implements SchemeFactory { - public GetTableTypes_resultStandardScheme getScheme() { - return new GetTableTypes_resultStandardScheme(); - } - } - - private static class GetTableTypes_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetTableTypesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetTableTypes_resultTupleSchemeFactory implements SchemeFactory { - public GetTableTypes_resultTupleScheme getScheme() { - return new GetTableTypes_resultTupleScheme(); - } - } - - private static class GetTableTypes_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetTableTypesResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetColumns_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetColumns_argsTupleSchemeFactory()); - } - - private TGetColumnsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_args.class, metaDataMap); - } - - public GetColumns_args() { - } - - public GetColumns_args( - TGetColumnsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetColumns_args(GetColumns_args other) { - if (other.isSetReq()) { - this.req = new TGetColumnsReq(other.req); - } - } - - public GetColumns_args deepCopy() { - return new GetColumns_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetColumnsReq getReq() { - return this.req; - } - - public void setReq(TGetColumnsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetColumnsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetColumns_args) - return this.equals((GetColumns_args)that); - return false; - } - - public boolean equals(GetColumns_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetColumns_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetColumns_args typedOther = (GetColumns_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetColumns_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetColumns_argsStandardSchemeFactory implements SchemeFactory { - public GetColumns_argsStandardScheme getScheme() { - return new GetColumns_argsStandardScheme(); - } - } - - private static class GetColumns_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetColumnsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetColumns_argsTupleSchemeFactory implements SchemeFactory { - public GetColumns_argsTupleScheme getScheme() { - return new GetColumns_argsTupleScheme(); - } - } - - private static class GetColumns_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetColumnsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetColumns_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetColumns_resultTupleSchemeFactory()); - } - - private TGetColumnsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_result.class, metaDataMap); - } - - public GetColumns_result() { - } - - public GetColumns_result( - TGetColumnsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetColumns_result(GetColumns_result other) { - if (other.isSetSuccess()) { - this.success = new TGetColumnsResp(other.success); - } - } - - public GetColumns_result deepCopy() { - return new GetColumns_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetColumnsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetColumnsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetColumnsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetColumns_result) - return this.equals((GetColumns_result)that); - return false; - } - - public boolean equals(GetColumns_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetColumns_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetColumns_result typedOther = (GetColumns_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetColumns_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetColumns_resultStandardSchemeFactory implements SchemeFactory { - public GetColumns_resultStandardScheme getScheme() { - return new GetColumns_resultStandardScheme(); - } - } - - private static class GetColumns_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetColumnsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetColumns_resultTupleSchemeFactory implements SchemeFactory { - public GetColumns_resultTupleScheme getScheme() { - return new GetColumns_resultTupleScheme(); - } - } - - private static class GetColumns_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetColumnsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetFunctions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetFunctions_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetFunctions_argsTupleSchemeFactory()); - } - - private TGetFunctionsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_args.class, metaDataMap); - } - - public GetFunctions_args() { - } - - public GetFunctions_args( - TGetFunctionsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetFunctions_args(GetFunctions_args other) { - if (other.isSetReq()) { - this.req = new TGetFunctionsReq(other.req); - } - } - - public GetFunctions_args deepCopy() { - return new GetFunctions_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetFunctionsReq getReq() { - return this.req; - } - - public void setReq(TGetFunctionsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetFunctionsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetFunctions_args) - return this.equals((GetFunctions_args)that); - return false; - } - - public boolean equals(GetFunctions_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetFunctions_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetFunctions_args typedOther = (GetFunctions_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetFunctions_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetFunctions_argsStandardSchemeFactory implements SchemeFactory { - public GetFunctions_argsStandardScheme getScheme() { - return new GetFunctions_argsStandardScheme(); - } - } - - private static class GetFunctions_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetFunctionsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetFunctions_argsTupleSchemeFactory implements SchemeFactory { - public GetFunctions_argsTupleScheme getScheme() { - return new GetFunctions_argsTupleScheme(); - } - } - - private static class GetFunctions_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetFunctionsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetFunctions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetFunctions_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetFunctions_resultTupleSchemeFactory()); - } - - private TGetFunctionsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_result.class, metaDataMap); - } - - public GetFunctions_result() { - } - - public GetFunctions_result( - TGetFunctionsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetFunctions_result(GetFunctions_result other) { - if (other.isSetSuccess()) { - this.success = new TGetFunctionsResp(other.success); - } - } - - public GetFunctions_result deepCopy() { - return new GetFunctions_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetFunctionsResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetFunctionsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetFunctionsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetFunctions_result) - return this.equals((GetFunctions_result)that); - return false; - } - - public boolean equals(GetFunctions_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetFunctions_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetFunctions_result typedOther = (GetFunctions_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetFunctions_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetFunctions_resultStandardSchemeFactory implements SchemeFactory { - public GetFunctions_resultStandardScheme getScheme() { - return new GetFunctions_resultStandardScheme(); - } - } - - private static class GetFunctions_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetFunctionsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetFunctions_resultTupleSchemeFactory implements SchemeFactory { - public GetFunctions_resultTupleScheme getScheme() { - return new GetFunctions_resultTupleScheme(); - } - } - - private static class GetFunctions_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetFunctionsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetOperationStatus_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetOperationStatus_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetOperationStatus_argsTupleSchemeFactory()); - } - - private TGetOperationStatusReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_args.class, metaDataMap); - } - - public GetOperationStatus_args() { - } - - public GetOperationStatus_args( - TGetOperationStatusReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetOperationStatus_args(GetOperationStatus_args other) { - if (other.isSetReq()) { - this.req = new TGetOperationStatusReq(other.req); - } - } - - public GetOperationStatus_args deepCopy() { - return new GetOperationStatus_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetOperationStatusReq getReq() { - return this.req; - } - - public void setReq(TGetOperationStatusReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetOperationStatusReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetOperationStatus_args) - return this.equals((GetOperationStatus_args)that); - return false; - } - - public boolean equals(GetOperationStatus_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetOperationStatus_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetOperationStatus_args typedOther = (GetOperationStatus_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetOperationStatus_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetOperationStatus_argsStandardSchemeFactory implements SchemeFactory { - public GetOperationStatus_argsStandardScheme getScheme() { - return new GetOperationStatus_argsStandardScheme(); - } - } - - private static class GetOperationStatus_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetOperationStatusReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetOperationStatus_argsTupleSchemeFactory implements SchemeFactory { - public GetOperationStatus_argsTupleScheme getScheme() { - return new GetOperationStatus_argsTupleScheme(); - } - } - - private static class GetOperationStatus_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetOperationStatusReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetOperationStatus_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetOperationStatus_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetOperationStatus_resultTupleSchemeFactory()); - } - - private TGetOperationStatusResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_result.class, metaDataMap); - } - - public GetOperationStatus_result() { - } - - public GetOperationStatus_result( - TGetOperationStatusResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetOperationStatus_result(GetOperationStatus_result other) { - if (other.isSetSuccess()) { - this.success = new TGetOperationStatusResp(other.success); - } - } - - public GetOperationStatus_result deepCopy() { - return new GetOperationStatus_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetOperationStatusResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetOperationStatusResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetOperationStatusResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetOperationStatus_result) - return this.equals((GetOperationStatus_result)that); - return false; - } - - public boolean equals(GetOperationStatus_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetOperationStatus_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetOperationStatus_result typedOther = (GetOperationStatus_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetOperationStatus_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetOperationStatus_resultStandardSchemeFactory implements SchemeFactory { - public GetOperationStatus_resultStandardScheme getScheme() { - return new GetOperationStatus_resultStandardScheme(); - } - } - - private static class GetOperationStatus_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetOperationStatusResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetOperationStatus_resultTupleSchemeFactory implements SchemeFactory { - public GetOperationStatus_resultTupleScheme getScheme() { - return new GetOperationStatus_resultTupleScheme(); - } - } - - private static class GetOperationStatus_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetOperationStatusResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CancelOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelOperation_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelOperation_argsTupleSchemeFactory()); - } - - private TCancelOperationReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_args.class, metaDataMap); - } - - public CancelOperation_args() { - } - - public CancelOperation_args( - TCancelOperationReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CancelOperation_args(CancelOperation_args other) { - if (other.isSetReq()) { - this.req = new TCancelOperationReq(other.req); - } - } - - public CancelOperation_args deepCopy() { - return new CancelOperation_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCancelOperationReq getReq() { - return this.req; - } - - public void setReq(TCancelOperationReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCancelOperationReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelOperation_args) - return this.equals((CancelOperation_args)that); - return false; - } - - public boolean equals(CancelOperation_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CancelOperation_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelOperation_args typedOther = (CancelOperation_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelOperation_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelOperation_argsStandardSchemeFactory implements SchemeFactory { - public CancelOperation_argsStandardScheme getScheme() { - return new CancelOperation_argsStandardScheme(); - } - } - - private static class CancelOperation_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCancelOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelOperation_argsTupleSchemeFactory implements SchemeFactory { - public CancelOperation_argsTupleScheme getScheme() { - return new CancelOperation_argsTupleScheme(); - } - } - - private static class CancelOperation_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCancelOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CancelOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelOperation_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelOperation_resultTupleSchemeFactory()); - } - - private TCancelOperationResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_result.class, metaDataMap); - } - - public CancelOperation_result() { - } - - public CancelOperation_result( - TCancelOperationResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CancelOperation_result(CancelOperation_result other) { - if (other.isSetSuccess()) { - this.success = new TCancelOperationResp(other.success); - } - } - - public CancelOperation_result deepCopy() { - return new CancelOperation_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCancelOperationResp getSuccess() { - return this.success; - } - - public void setSuccess(TCancelOperationResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCancelOperationResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelOperation_result) - return this.equals((CancelOperation_result)that); - return false; - } - - public boolean equals(CancelOperation_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CancelOperation_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelOperation_result typedOther = (CancelOperation_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelOperation_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelOperation_resultStandardSchemeFactory implements SchemeFactory { - public CancelOperation_resultStandardScheme getScheme() { - return new CancelOperation_resultStandardScheme(); - } - } - - private static class CancelOperation_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCancelOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelOperation_resultTupleSchemeFactory implements SchemeFactory { - public CancelOperation_resultTupleScheme getScheme() { - return new CancelOperation_resultTupleScheme(); - } - } - - private static class CancelOperation_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCancelOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CloseOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseOperation_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseOperation_argsTupleSchemeFactory()); - } - - private TCloseOperationReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_args.class, metaDataMap); - } - - public CloseOperation_args() { - } - - public CloseOperation_args( - TCloseOperationReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CloseOperation_args(CloseOperation_args other) { - if (other.isSetReq()) { - this.req = new TCloseOperationReq(other.req); - } - } - - public CloseOperation_args deepCopy() { - return new CloseOperation_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCloseOperationReq getReq() { - return this.req; - } - - public void setReq(TCloseOperationReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCloseOperationReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseOperation_args) - return this.equals((CloseOperation_args)that); - return false; - } - - public boolean equals(CloseOperation_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CloseOperation_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseOperation_args typedOther = (CloseOperation_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseOperation_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseOperation_argsStandardSchemeFactory implements SchemeFactory { - public CloseOperation_argsStandardScheme getScheme() { - return new CloseOperation_argsStandardScheme(); - } - } - - private static class CloseOperation_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCloseOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseOperation_argsTupleSchemeFactory implements SchemeFactory { - public CloseOperation_argsTupleScheme getScheme() { - return new CloseOperation_argsTupleScheme(); - } - } - - private static class CloseOperation_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCloseOperationReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CloseOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CloseOperation_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CloseOperation_resultTupleSchemeFactory()); - } - - private TCloseOperationResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_result.class, metaDataMap); - } - - public CloseOperation_result() { - } - - public CloseOperation_result( - TCloseOperationResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CloseOperation_result(CloseOperation_result other) { - if (other.isSetSuccess()) { - this.success = new TCloseOperationResp(other.success); - } - } - - public CloseOperation_result deepCopy() { - return new CloseOperation_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCloseOperationResp getSuccess() { - return this.success; - } - - public void setSuccess(TCloseOperationResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCloseOperationResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CloseOperation_result) - return this.equals((CloseOperation_result)that); - return false; - } - - public boolean equals(CloseOperation_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CloseOperation_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CloseOperation_result typedOther = (CloseOperation_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CloseOperation_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CloseOperation_resultStandardSchemeFactory implements SchemeFactory { - public CloseOperation_resultStandardScheme getScheme() { - return new CloseOperation_resultStandardScheme(); - } - } - - private static class CloseOperation_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCloseOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CloseOperation_resultTupleSchemeFactory implements SchemeFactory { - public CloseOperation_resultTupleScheme getScheme() { - return new CloseOperation_resultTupleScheme(); - } - } - - private static class CloseOperation_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCloseOperationResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetResultSetMetadata_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetResultSetMetadata_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetResultSetMetadata_argsTupleSchemeFactory()); - } - - private TGetResultSetMetadataReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_args.class, metaDataMap); - } - - public GetResultSetMetadata_args() { - } - - public GetResultSetMetadata_args( - TGetResultSetMetadataReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetResultSetMetadata_args(GetResultSetMetadata_args other) { - if (other.isSetReq()) { - this.req = new TGetResultSetMetadataReq(other.req); - } - } - - public GetResultSetMetadata_args deepCopy() { - return new GetResultSetMetadata_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetResultSetMetadataReq getReq() { - return this.req; - } - - public void setReq(TGetResultSetMetadataReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetResultSetMetadataReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetResultSetMetadata_args) - return this.equals((GetResultSetMetadata_args)that); - return false; - } - - public boolean equals(GetResultSetMetadata_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetResultSetMetadata_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetResultSetMetadata_args typedOther = (GetResultSetMetadata_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetResultSetMetadata_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetResultSetMetadata_argsStandardSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_argsStandardScheme getScheme() { - return new GetResultSetMetadata_argsStandardScheme(); - } - } - - private static class GetResultSetMetadata_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetResultSetMetadataReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetResultSetMetadata_argsTupleSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_argsTupleScheme getScheme() { - return new GetResultSetMetadata_argsTupleScheme(); - } - } - - private static class GetResultSetMetadata_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetResultSetMetadataReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetResultSetMetadata_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetResultSetMetadata_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetResultSetMetadata_resultTupleSchemeFactory()); - } - - private TGetResultSetMetadataResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_result.class, metaDataMap); - } - - public GetResultSetMetadata_result() { - } - - public GetResultSetMetadata_result( - TGetResultSetMetadataResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetResultSetMetadata_result(GetResultSetMetadata_result other) { - if (other.isSetSuccess()) { - this.success = new TGetResultSetMetadataResp(other.success); - } - } - - public GetResultSetMetadata_result deepCopy() { - return new GetResultSetMetadata_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetResultSetMetadataResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetResultSetMetadataResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetResultSetMetadataResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetResultSetMetadata_result) - return this.equals((GetResultSetMetadata_result)that); - return false; - } - - public boolean equals(GetResultSetMetadata_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetResultSetMetadata_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetResultSetMetadata_result typedOther = (GetResultSetMetadata_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetResultSetMetadata_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetResultSetMetadata_resultStandardSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_resultStandardScheme getScheme() { - return new GetResultSetMetadata_resultStandardScheme(); - } - } - - private static class GetResultSetMetadata_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetResultSetMetadataResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetResultSetMetadata_resultTupleSchemeFactory implements SchemeFactory { - public GetResultSetMetadata_resultTupleScheme getScheme() { - return new GetResultSetMetadata_resultTupleScheme(); - } - } - - private static class GetResultSetMetadata_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetResultSetMetadataResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class FetchResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new FetchResults_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new FetchResults_argsTupleSchemeFactory()); - } - - private TFetchResultsReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_args.class, metaDataMap); - } - - public FetchResults_args() { - } - - public FetchResults_args( - TFetchResultsReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public FetchResults_args(FetchResults_args other) { - if (other.isSetReq()) { - this.req = new TFetchResultsReq(other.req); - } - } - - public FetchResults_args deepCopy() { - return new FetchResults_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TFetchResultsReq getReq() { - return this.req; - } - - public void setReq(TFetchResultsReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TFetchResultsReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof FetchResults_args) - return this.equals((FetchResults_args)that); - return false; - } - - public boolean equals(FetchResults_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(FetchResults_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - FetchResults_args typedOther = (FetchResults_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("FetchResults_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class FetchResults_argsStandardSchemeFactory implements SchemeFactory { - public FetchResults_argsStandardScheme getScheme() { - return new FetchResults_argsStandardScheme(); - } - } - - private static class FetchResults_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TFetchResultsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class FetchResults_argsTupleSchemeFactory implements SchemeFactory { - public FetchResults_argsTupleScheme getScheme() { - return new FetchResults_argsTupleScheme(); - } - } - - private static class FetchResults_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TFetchResultsReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class FetchResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new FetchResults_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new FetchResults_resultTupleSchemeFactory()); - } - - private TFetchResultsResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_result.class, metaDataMap); - } - - public FetchResults_result() { - } - - public FetchResults_result( - TFetchResultsResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public FetchResults_result(FetchResults_result other) { - if (other.isSetSuccess()) { - this.success = new TFetchResultsResp(other.success); - } - } - - public FetchResults_result deepCopy() { - return new FetchResults_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TFetchResultsResp getSuccess() { - return this.success; - } - - public void setSuccess(TFetchResultsResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TFetchResultsResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof FetchResults_result) - return this.equals((FetchResults_result)that); - return false; - } - - public boolean equals(FetchResults_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(FetchResults_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - FetchResults_result typedOther = (FetchResults_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("FetchResults_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class FetchResults_resultStandardSchemeFactory implements SchemeFactory { - public FetchResults_resultStandardScheme getScheme() { - return new FetchResults_resultStandardScheme(); - } - } - - private static class FetchResults_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TFetchResultsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class FetchResults_resultTupleSchemeFactory implements SchemeFactory { - public FetchResults_resultTupleScheme getScheme() { - return new FetchResults_resultTupleScheme(); - } - } - - private static class FetchResults_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TFetchResultsResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class GetDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetDelegationToken_argsTupleSchemeFactory()); - } - - private TGetDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_args.class, metaDataMap); - } - - public GetDelegationToken_args() { - } - - public GetDelegationToken_args( - TGetDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public GetDelegationToken_args(GetDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TGetDelegationTokenReq(other.req); - } - } - - public GetDelegationToken_args deepCopy() { - return new GetDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TGetDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TGetDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TGetDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetDelegationToken_args) - return this.equals((GetDelegationToken_args)that); - return false; - } - - public boolean equals(GetDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(GetDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetDelegationToken_args typedOther = (GetDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public GetDelegationToken_argsStandardScheme getScheme() { - return new GetDelegationToken_argsStandardScheme(); - } - } - - private static class GetDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TGetDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public GetDelegationToken_argsTupleScheme getScheme() { - return new GetDelegationToken_argsTupleScheme(); - } - } - - private static class GetDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TGetDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class GetDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new GetDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new GetDelegationToken_resultTupleSchemeFactory()); - } - - private TGetDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_result.class, metaDataMap); - } - - public GetDelegationToken_result() { - } - - public GetDelegationToken_result( - TGetDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public GetDelegationToken_result(GetDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TGetDelegationTokenResp(other.success); - } - } - - public GetDelegationToken_result deepCopy() { - return new GetDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TGetDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TGetDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TGetDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GetDelegationToken_result) - return this.equals((GetDelegationToken_result)that); - return false; - } - - public boolean equals(GetDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(GetDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GetDelegationToken_result typedOther = (GetDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GetDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class GetDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public GetDelegationToken_resultStandardScheme getScheme() { - return new GetDelegationToken_resultStandardScheme(); - } - } - - private static class GetDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TGetDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class GetDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public GetDelegationToken_resultTupleScheme getScheme() { - return new GetDelegationToken_resultTupleScheme(); - } - } - - private static class GetDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TGetDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class CancelDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelDelegationToken_argsTupleSchemeFactory()); - } - - private TCancelDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_args.class, metaDataMap); - } - - public CancelDelegationToken_args() { - } - - public CancelDelegationToken_args( - TCancelDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public CancelDelegationToken_args(CancelDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TCancelDelegationTokenReq(other.req); - } - } - - public CancelDelegationToken_args deepCopy() { - return new CancelDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TCancelDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TCancelDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TCancelDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelDelegationToken_args) - return this.equals((CancelDelegationToken_args)that); - return false; - } - - public boolean equals(CancelDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(CancelDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelDelegationToken_args typedOther = (CancelDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public CancelDelegationToken_argsStandardScheme getScheme() { - return new CancelDelegationToken_argsStandardScheme(); - } - } - - private static class CancelDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TCancelDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public CancelDelegationToken_argsTupleScheme getScheme() { - return new CancelDelegationToken_argsTupleScheme(); - } - } - - private static class CancelDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TCancelDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class CancelDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new CancelDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new CancelDelegationToken_resultTupleSchemeFactory()); - } - - private TCancelDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_result.class, metaDataMap); - } - - public CancelDelegationToken_result() { - } - - public CancelDelegationToken_result( - TCancelDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public CancelDelegationToken_result(CancelDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TCancelDelegationTokenResp(other.success); - } - } - - public CancelDelegationToken_result deepCopy() { - return new CancelDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TCancelDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TCancelDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TCancelDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof CancelDelegationToken_result) - return this.equals((CancelDelegationToken_result)that); - return false; - } - - public boolean equals(CancelDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(CancelDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - CancelDelegationToken_result typedOther = (CancelDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("CancelDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class CancelDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public CancelDelegationToken_resultStandardScheme getScheme() { - return new CancelDelegationToken_resultStandardScheme(); - } - } - - private static class CancelDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TCancelDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class CancelDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public CancelDelegationToken_resultTupleScheme getScheme() { - return new CancelDelegationToken_resultTupleScheme(); - } - } - - private static class CancelDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TCancelDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class RenewDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_args"); - - private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new RenewDelegationToken_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new RenewDelegationToken_argsTupleSchemeFactory()); - } - - private TRenewDelegationTokenReq req; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - REQ((short)1, "req"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // REQ - return REQ; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenReq.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_args.class, metaDataMap); - } - - public RenewDelegationToken_args() { - } - - public RenewDelegationToken_args( - TRenewDelegationTokenReq req) - { - this(); - this.req = req; - } - - /** - * Performs a deep copy on other. - */ - public RenewDelegationToken_args(RenewDelegationToken_args other) { - if (other.isSetReq()) { - this.req = new TRenewDelegationTokenReq(other.req); - } - } - - public RenewDelegationToken_args deepCopy() { - return new RenewDelegationToken_args(this); - } - - @Override - public void clear() { - this.req = null; - } - - public TRenewDelegationTokenReq getReq() { - return this.req; - } - - public void setReq(TRenewDelegationTokenReq req) { - this.req = req; - } - - public void unsetReq() { - this.req = null; - } - - /** Returns true if field req is set (has been assigned a value) and false otherwise */ - public boolean isSetReq() { - return this.req != null; - } - - public void setReqIsSet(boolean value) { - if (!value) { - this.req = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case REQ: - if (value == null) { - unsetReq(); - } else { - setReq((TRenewDelegationTokenReq)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case REQ: - return getReq(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case REQ: - return isSetReq(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof RenewDelegationToken_args) - return this.equals((RenewDelegationToken_args)that); - return false; - } - - public boolean equals(RenewDelegationToken_args that) { - if (that == null) - return false; - - boolean this_present_req = true && this.isSetReq(); - boolean that_present_req = true && that.isSetReq(); - if (this_present_req || that_present_req) { - if (!(this_present_req && that_present_req)) - return false; - if (!this.req.equals(that.req)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_req = true && (isSetReq()); - builder.append(present_req); - if (present_req) - builder.append(req); - - return builder.toHashCode(); - } - - public int compareTo(RenewDelegationToken_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - RenewDelegationToken_args typedOther = (RenewDelegationToken_args)other; - - lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReq()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("RenewDelegationToken_args("); - boolean first = true; - - sb.append("req:"); - if (this.req == null) { - sb.append("null"); - } else { - sb.append(this.req); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (req != null) { - req.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class RenewDelegationToken_argsStandardSchemeFactory implements SchemeFactory { - public RenewDelegationToken_argsStandardScheme getScheme() { - return new RenewDelegationToken_argsStandardScheme(); - } - } - - private static class RenewDelegationToken_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // REQ - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.req = new TRenewDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.req != null) { - oprot.writeFieldBegin(REQ_FIELD_DESC); - struct.req.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class RenewDelegationToken_argsTupleSchemeFactory implements SchemeFactory { - public RenewDelegationToken_argsTupleScheme getScheme() { - return new RenewDelegationToken_argsTupleScheme(); - } - } - - private static class RenewDelegationToken_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetReq()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetReq()) { - struct.req.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.req = new TRenewDelegationTokenReq(); - struct.req.read(iprot); - struct.setReqIsSet(true); - } - } - } - - } - - public static class RenewDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new RenewDelegationToken_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new RenewDelegationToken_resultTupleSchemeFactory()); - } - - private TRenewDelegationTokenResp success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenResp.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_result.class, metaDataMap); - } - - public RenewDelegationToken_result() { - } - - public RenewDelegationToken_result( - TRenewDelegationTokenResp success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public RenewDelegationToken_result(RenewDelegationToken_result other) { - if (other.isSetSuccess()) { - this.success = new TRenewDelegationTokenResp(other.success); - } - } - - public RenewDelegationToken_result deepCopy() { - return new RenewDelegationToken_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public TRenewDelegationTokenResp getSuccess() { - return this.success; - } - - public void setSuccess(TRenewDelegationTokenResp success) { - this.success = success; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((TRenewDelegationTokenResp)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof RenewDelegationToken_result) - return this.equals((RenewDelegationToken_result)that); - return false; - } - - public boolean equals(RenewDelegationToken_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (isSetSuccess()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(RenewDelegationToken_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - RenewDelegationToken_result typedOther = (RenewDelegationToken_result)other; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("RenewDelegationToken_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class RenewDelegationToken_resultStandardSchemeFactory implements SchemeFactory { - public RenewDelegationToken_resultStandardScheme getScheme() { - return new RenewDelegationToken_resultStandardScheme(); - } - } - - private static class RenewDelegationToken_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new TRenewDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class RenewDelegationToken_resultTupleSchemeFactory implements SchemeFactory { - public RenewDelegationToken_resultTupleScheme getScheme() { - return new RenewDelegationToken_resultTupleScheme(); - } - } - - private static class RenewDelegationToken_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = new TRenewDelegationTokenResp(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - } - } - - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java deleted file mode 100644 index 25a38b178428a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java +++ /dev/null @@ -1,103 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCLIServiceConstants { - - public static final Set PRIMITIVE_TYPES = new HashSet(); - static { - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE); - PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE); - } - - public static final Set COMPLEX_TYPES = new HashSet(); - static { - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE); - COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.USER_DEFINED_TYPE); - } - - public static final Set COLLECTION_TYPES = new HashSet(); - static { - COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); - COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); - } - - public static final Map TYPE_NAMES = new HashMap(); - static { - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE, "BOOLEAN"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE, "TINYINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE, "SMALLINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE, "INT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE, "BIGINT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE, "FLOAT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE, "DOUBLE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE, "STRING"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE, "TIMESTAMP"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE, "BINARY"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE, "ARRAY"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE, "MAP"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE, "STRUCT"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE, "UNIONTYPE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE, "DECIMAL"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE, "NULL"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE, "DATE"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE, "VARCHAR"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE, "CHAR"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE, "INTERVAL_YEAR_MONTH"); - TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE, "INTERVAL_DAY_TIME"); - } - - public static final String CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"; - - public static final String PRECISION = "precision"; - - public static final String SCALE = "scale"; - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java deleted file mode 100644 index e23fcdd77a1a4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java +++ /dev/null @@ -1,491 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String delegationToken; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenReq.class, metaDataMap); - } - - public TCancelDelegationTokenReq() { - } - - public TCancelDelegationTokenReq( - TSessionHandle sessionHandle, - String delegationToken) - { - this(); - this.sessionHandle = sessionHandle; - this.delegationToken = delegationToken; - } - - /** - * Performs a deep copy on other. - */ - public TCancelDelegationTokenReq(TCancelDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TCancelDelegationTokenReq deepCopy() { - return new TCancelDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.delegationToken = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelDelegationTokenReq) - return this.equals((TCancelDelegationTokenReq)that); - return false; - } - - public boolean equals(TCancelDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TCancelDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelDelegationTokenReq typedOther = (TCancelDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetDelegationToken()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenReqStandardScheme getScheme() { - return new TCancelDelegationTokenReqStandardScheme(); - } - } - - private static class TCancelDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenReqTupleScheme getScheme() { - return new TCancelDelegationTokenReqTupleScheme(); - } - } - - private static class TCancelDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.delegationToken); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java deleted file mode 100644 index 77c9ee77ec59b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenResp.class, metaDataMap); - } - - public TCancelDelegationTokenResp() { - } - - public TCancelDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCancelDelegationTokenResp(TCancelDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCancelDelegationTokenResp deepCopy() { - return new TCancelDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelDelegationTokenResp) - return this.equals((TCancelDelegationTokenResp)that); - return false; - } - - public boolean equals(TCancelDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCancelDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelDelegationTokenResp typedOther = (TCancelDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenRespStandardScheme getScheme() { - return new TCancelDelegationTokenRespStandardScheme(); - } - } - - private static class TCancelDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TCancelDelegationTokenRespTupleScheme getScheme() { - return new TCancelDelegationTokenRespTupleScheme(); - } - } - - private static class TCancelDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java deleted file mode 100644 index 45eac48ab12d3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelOperationReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelOperationReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationReq.class, metaDataMap); - } - - public TCancelOperationReq() { - } - - public TCancelOperationReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCancelOperationReq(TCancelOperationReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TCancelOperationReq deepCopy() { - return new TCancelOperationReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelOperationReq) - return this.equals((TCancelOperationReq)that); - return false; - } - - public boolean equals(TCancelOperationReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCancelOperationReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelOperationReq typedOther = (TCancelOperationReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelOperationReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelOperationReqStandardSchemeFactory implements SchemeFactory { - public TCancelOperationReqStandardScheme getScheme() { - return new TCancelOperationReqStandardScheme(); - } - } - - private static class TCancelOperationReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelOperationReqTupleSchemeFactory implements SchemeFactory { - public TCancelOperationReqTupleScheme getScheme() { - return new TCancelOperationReqTupleScheme(); - } - } - - private static class TCancelOperationReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java deleted file mode 100644 index 2a39414d601aa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCancelOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCancelOperationRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCancelOperationRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationResp.class, metaDataMap); - } - - public TCancelOperationResp() { - } - - public TCancelOperationResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCancelOperationResp(TCancelOperationResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCancelOperationResp deepCopy() { - return new TCancelOperationResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCancelOperationResp) - return this.equals((TCancelOperationResp)that); - return false; - } - - public boolean equals(TCancelOperationResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCancelOperationResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCancelOperationResp typedOther = (TCancelOperationResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCancelOperationResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCancelOperationRespStandardSchemeFactory implements SchemeFactory { - public TCancelOperationRespStandardScheme getScheme() { - return new TCancelOperationRespStandardScheme(); - } - } - - private static class TCancelOperationRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCancelOperationRespTupleSchemeFactory implements SchemeFactory { - public TCancelOperationRespTupleScheme getScheme() { - return new TCancelOperationRespTupleScheme(); - } - } - - private static class TCancelOperationRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java deleted file mode 100644 index 0cbb7ccced073..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseOperationReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseOperationReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationReq.class, metaDataMap); - } - - public TCloseOperationReq() { - } - - public TCloseOperationReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCloseOperationReq(TCloseOperationReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TCloseOperationReq deepCopy() { - return new TCloseOperationReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseOperationReq) - return this.equals((TCloseOperationReq)that); - return false; - } - - public boolean equals(TCloseOperationReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCloseOperationReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseOperationReq typedOther = (TCloseOperationReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseOperationReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseOperationReqStandardSchemeFactory implements SchemeFactory { - public TCloseOperationReqStandardScheme getScheme() { - return new TCloseOperationReqStandardScheme(); - } - } - - private static class TCloseOperationReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseOperationReqTupleSchemeFactory implements SchemeFactory { - public TCloseOperationReqTupleScheme getScheme() { - return new TCloseOperationReqTupleScheme(); - } - } - - private static class TCloseOperationReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java deleted file mode 100644 index 7334d67173d7b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseOperationRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseOperationRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationResp.class, metaDataMap); - } - - public TCloseOperationResp() { - } - - public TCloseOperationResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCloseOperationResp(TCloseOperationResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCloseOperationResp deepCopy() { - return new TCloseOperationResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseOperationResp) - return this.equals((TCloseOperationResp)that); - return false; - } - - public boolean equals(TCloseOperationResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCloseOperationResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseOperationResp typedOther = (TCloseOperationResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseOperationResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseOperationRespStandardSchemeFactory implements SchemeFactory { - public TCloseOperationRespStandardScheme getScheme() { - return new TCloseOperationRespStandardScheme(); - } - } - - private static class TCloseOperationRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseOperationRespTupleSchemeFactory implements SchemeFactory { - public TCloseOperationRespTupleScheme getScheme() { - return new TCloseOperationRespTupleScheme(); - } - } - - private static class TCloseOperationRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java deleted file mode 100644 index 027e8295436b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseSessionReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseSessionReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionReq.class, metaDataMap); - } - - public TCloseSessionReq() { - } - - public TCloseSessionReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TCloseSessionReq(TCloseSessionReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TCloseSessionReq deepCopy() { - return new TCloseSessionReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseSessionReq) - return this.equals((TCloseSessionReq)that); - return false; - } - - public boolean equals(TCloseSessionReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TCloseSessionReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseSessionReq typedOther = (TCloseSessionReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseSessionReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseSessionReqStandardSchemeFactory implements SchemeFactory { - public TCloseSessionReqStandardScheme getScheme() { - return new TCloseSessionReqStandardScheme(); - } - } - - private static class TCloseSessionReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseSessionReqTupleSchemeFactory implements SchemeFactory { - public TCloseSessionReqTupleScheme getScheme() { - return new TCloseSessionReqTupleScheme(); - } - } - - private static class TCloseSessionReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java deleted file mode 100644 index 168c8fc775e33..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TCloseSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TCloseSessionRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TCloseSessionRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionResp.class, metaDataMap); - } - - public TCloseSessionResp() { - } - - public TCloseSessionResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TCloseSessionResp(TCloseSessionResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TCloseSessionResp deepCopy() { - return new TCloseSessionResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TCloseSessionResp) - return this.equals((TCloseSessionResp)that); - return false; - } - - public boolean equals(TCloseSessionResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TCloseSessionResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TCloseSessionResp typedOther = (TCloseSessionResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TCloseSessionResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TCloseSessionRespStandardSchemeFactory implements SchemeFactory { - public TCloseSessionRespStandardScheme getScheme() { - return new TCloseSessionRespStandardScheme(); - } - } - - private static class TCloseSessionRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TCloseSessionRespTupleSchemeFactory implements SchemeFactory { - public TCloseSessionRespTupleScheme getScheme() { - return new TCloseSessionRespTupleScheme(); - } - } - - private static class TCloseSessionRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java deleted file mode 100644 index fc2171dc99e4c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java +++ /dev/null @@ -1,732 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumn extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); - private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); - private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); - private static final org.apache.thrift.protocol.TField BINARY_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryVal", org.apache.thrift.protocol.TType.STRUCT, (short)8); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - BOOL_VAL((short)1, "boolVal"), - BYTE_VAL((short)2, "byteVal"), - I16_VAL((short)3, "i16Val"), - I32_VAL((short)4, "i32Val"), - I64_VAL((short)5, "i64Val"), - DOUBLE_VAL((short)6, "doubleVal"), - STRING_VAL((short)7, "stringVal"), - BINARY_VAL((short)8, "binaryVal"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOOL_VAL - return BOOL_VAL; - case 2: // BYTE_VAL - return BYTE_VAL; - case 3: // I16_VAL - return I16_VAL; - case 4: // I32_VAL - return I32_VAL; - case 5: // I64_VAL - return I64_VAL; - case 6: // DOUBLE_VAL - return DOUBLE_VAL; - case 7: // STRING_VAL - return STRING_VAL; - case 8: // BINARY_VAL - return BINARY_VAL; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolColumn.class))); - tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteColumn.class))); - tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Column.class))); - tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Column.class))); - tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Column.class))); - tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleColumn.class))); - tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringColumn.class))); - tmpMap.put(_Fields.BINARY_VAL, new org.apache.thrift.meta_data.FieldMetaData("binaryVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBinaryColumn.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap); - } - - public TColumn() { - super(); - } - - public TColumn(TColumn._Fields setField, Object value) { - super(setField, value); - } - - public TColumn(TColumn other) { - super(other); - } - public TColumn deepCopy() { - return new TColumn(this); - } - - public static TColumn boolVal(TBoolColumn value) { - TColumn x = new TColumn(); - x.setBoolVal(value); - return x; - } - - public static TColumn byteVal(TByteColumn value) { - TColumn x = new TColumn(); - x.setByteVal(value); - return x; - } - - public static TColumn i16Val(TI16Column value) { - TColumn x = new TColumn(); - x.setI16Val(value); - return x; - } - - public static TColumn i32Val(TI32Column value) { - TColumn x = new TColumn(); - x.setI32Val(value); - return x; - } - - public static TColumn i64Val(TI64Column value) { - TColumn x = new TColumn(); - x.setI64Val(value); - return x; - } - - public static TColumn doubleVal(TDoubleColumn value) { - TColumn x = new TColumn(); - x.setDoubleVal(value); - return x; - } - - public static TColumn stringVal(TStringColumn value) { - TColumn x = new TColumn(); - x.setStringVal(value); - return x; - } - - public static TColumn binaryVal(TBinaryColumn value) { - TColumn x = new TColumn(); - x.setBinaryVal(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case BOOL_VAL: - if (value instanceof TBoolColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TBoolColumn for field 'boolVal', but got " + value.getClass().getSimpleName()); - case BYTE_VAL: - if (value instanceof TByteColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TByteColumn for field 'byteVal', but got " + value.getClass().getSimpleName()); - case I16_VAL: - if (value instanceof TI16Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI16Column for field 'i16Val', but got " + value.getClass().getSimpleName()); - case I32_VAL: - if (value instanceof TI32Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI32Column for field 'i32Val', but got " + value.getClass().getSimpleName()); - case I64_VAL: - if (value instanceof TI64Column) { - break; - } - throw new ClassCastException("Was expecting value of type TI64Column for field 'i64Val', but got " + value.getClass().getSimpleName()); - case DOUBLE_VAL: - if (value instanceof TDoubleColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TDoubleColumn for field 'doubleVal', but got " + value.getClass().getSimpleName()); - case STRING_VAL: - if (value instanceof TStringColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TStringColumn for field 'stringVal', but got " + value.getClass().getSimpleName()); - case BINARY_VAL: - if (value instanceof TBinaryColumn) { - break; - } - throw new ClassCastException("Was expecting value of type TBinaryColumn for field 'binaryVal', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - if (field.type == BOOL_VAL_FIELD_DESC.type) { - TBoolColumn boolVal; - boolVal = new TBoolColumn(); - boolVal.read(iprot); - return boolVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BYTE_VAL: - if (field.type == BYTE_VAL_FIELD_DESC.type) { - TByteColumn byteVal; - byteVal = new TByteColumn(); - byteVal.read(iprot); - return byteVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I16_VAL: - if (field.type == I16_VAL_FIELD_DESC.type) { - TI16Column i16Val; - i16Val = new TI16Column(); - i16Val.read(iprot); - return i16Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I32_VAL: - if (field.type == I32_VAL_FIELD_DESC.type) { - TI32Column i32Val; - i32Val = new TI32Column(); - i32Val.read(iprot); - return i32Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I64_VAL: - if (field.type == I64_VAL_FIELD_DESC.type) { - TI64Column i64Val; - i64Val = new TI64Column(); - i64Val.read(iprot); - return i64Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DOUBLE_VAL: - if (field.type == DOUBLE_VAL_FIELD_DESC.type) { - TDoubleColumn doubleVal; - doubleVal = new TDoubleColumn(); - doubleVal.read(iprot); - return doubleVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VAL: - if (field.type == STRING_VAL_FIELD_DESC.type) { - TStringColumn stringVal; - stringVal = new TStringColumn(); - stringVal.read(iprot); - return stringVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BINARY_VAL: - if (field.type == BINARY_VAL_FIELD_DESC.type) { - TBinaryColumn binaryVal; - binaryVal = new TBinaryColumn(); - binaryVal.read(iprot); - return binaryVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolColumn boolVal = (TBoolColumn)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteColumn byteVal = (TByteColumn)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Column i16Val = (TI16Column)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Column i32Val = (TI32Column)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Column i64Val = (TI64Column)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleColumn doubleVal = (TDoubleColumn)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringColumn stringVal = (TStringColumn)value_; - stringVal.write(oprot); - return; - case BINARY_VAL: - TBinaryColumn binaryVal = (TBinaryColumn)value_; - binaryVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - TBoolColumn boolVal; - boolVal = new TBoolColumn(); - boolVal.read(iprot); - return boolVal; - case BYTE_VAL: - TByteColumn byteVal; - byteVal = new TByteColumn(); - byteVal.read(iprot); - return byteVal; - case I16_VAL: - TI16Column i16Val; - i16Val = new TI16Column(); - i16Val.read(iprot); - return i16Val; - case I32_VAL: - TI32Column i32Val; - i32Val = new TI32Column(); - i32Val.read(iprot); - return i32Val; - case I64_VAL: - TI64Column i64Val; - i64Val = new TI64Column(); - i64Val.read(iprot); - return i64Val; - case DOUBLE_VAL: - TDoubleColumn doubleVal; - doubleVal = new TDoubleColumn(); - doubleVal.read(iprot); - return doubleVal; - case STRING_VAL: - TStringColumn stringVal; - stringVal = new TStringColumn(); - stringVal.read(iprot); - return stringVal; - case BINARY_VAL: - TBinaryColumn binaryVal; - binaryVal = new TBinaryColumn(); - binaryVal.read(iprot); - return binaryVal; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolColumn boolVal = (TBoolColumn)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteColumn byteVal = (TByteColumn)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Column i16Val = (TI16Column)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Column i32Val = (TI32Column)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Column i64Val = (TI64Column)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleColumn doubleVal = (TDoubleColumn)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringColumn stringVal = (TStringColumn)value_; - stringVal.write(oprot); - return; - case BINARY_VAL: - TBinaryColumn binaryVal = (TBinaryColumn)value_; - binaryVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case BOOL_VAL: - return BOOL_VAL_FIELD_DESC; - case BYTE_VAL: - return BYTE_VAL_FIELD_DESC; - case I16_VAL: - return I16_VAL_FIELD_DESC; - case I32_VAL: - return I32_VAL_FIELD_DESC; - case I64_VAL: - return I64_VAL_FIELD_DESC; - case DOUBLE_VAL: - return DOUBLE_VAL_FIELD_DESC; - case STRING_VAL: - return STRING_VAL_FIELD_DESC; - case BINARY_VAL: - return BINARY_VAL_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TBoolColumn getBoolVal() { - if (getSetField() == _Fields.BOOL_VAL) { - return (TBoolColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBoolVal(TBoolColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BOOL_VAL; - value_ = value; - } - - public TByteColumn getByteVal() { - if (getSetField() == _Fields.BYTE_VAL) { - return (TByteColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setByteVal(TByteColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BYTE_VAL; - value_ = value; - } - - public TI16Column getI16Val() { - if (getSetField() == _Fields.I16_VAL) { - return (TI16Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI16Val(TI16Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I16_VAL; - value_ = value; - } - - public TI32Column getI32Val() { - if (getSetField() == _Fields.I32_VAL) { - return (TI32Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Val(TI32Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I32_VAL; - value_ = value; - } - - public TI64Column getI64Val() { - if (getSetField() == _Fields.I64_VAL) { - return (TI64Column)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI64Val(TI64Column value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I64_VAL; - value_ = value; - } - - public TDoubleColumn getDoubleVal() { - if (getSetField() == _Fields.DOUBLE_VAL) { - return (TDoubleColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setDoubleVal(TDoubleColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.DOUBLE_VAL; - value_ = value; - } - - public TStringColumn getStringVal() { - if (getSetField() == _Fields.STRING_VAL) { - return (TStringColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringVal(TStringColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VAL; - value_ = value; - } - - public TBinaryColumn getBinaryVal() { - if (getSetField() == _Fields.BINARY_VAL) { - return (TBinaryColumn)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'binaryVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBinaryVal(TBinaryColumn value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BINARY_VAL; - value_ = value; - } - - public boolean isSetBoolVal() { - return setField_ == _Fields.BOOL_VAL; - } - - - public boolean isSetByteVal() { - return setField_ == _Fields.BYTE_VAL; - } - - - public boolean isSetI16Val() { - return setField_ == _Fields.I16_VAL; - } - - - public boolean isSetI32Val() { - return setField_ == _Fields.I32_VAL; - } - - - public boolean isSetI64Val() { - return setField_ == _Fields.I64_VAL; - } - - - public boolean isSetDoubleVal() { - return setField_ == _Fields.DOUBLE_VAL; - } - - - public boolean isSetStringVal() { - return setField_ == _Fields.STRING_VAL; - } - - - public boolean isSetBinaryVal() { - return setField_ == _Fields.BINARY_VAL; - } - - - public boolean equals(Object other) { - if (other instanceof TColumn) { - return equals((TColumn)other); - } else { - return false; - } - } - - public boolean equals(TColumn other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TColumn other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java deleted file mode 100644 index 247db6489457f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java +++ /dev/null @@ -1,700 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumnDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnDesc"); - - private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField TYPE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("typeDesc", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField POSITION_FIELD_DESC = new org.apache.thrift.protocol.TField("position", org.apache.thrift.protocol.TType.I32, (short)3); - private static final org.apache.thrift.protocol.TField COMMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("comment", org.apache.thrift.protocol.TType.STRING, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TColumnDescStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TColumnDescTupleSchemeFactory()); - } - - private String columnName; // required - private TTypeDesc typeDesc; // required - private int position; // required - private String comment; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COLUMN_NAME((short)1, "columnName"), - TYPE_DESC((short)2, "typeDesc"), - POSITION((short)3, "position"), - COMMENT((short)4, "comment"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COLUMN_NAME - return COLUMN_NAME; - case 2: // TYPE_DESC - return TYPE_DESC; - case 3: // POSITION - return POSITION; - case 4: // COMMENT - return COMMENT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __POSITION_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.COMMENT}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.TYPE_DESC, new org.apache.thrift.meta_data.FieldMetaData("typeDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeDesc.class))); - tmpMap.put(_Fields.POSITION, new org.apache.thrift.meta_data.FieldMetaData("position", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.COMMENT, new org.apache.thrift.meta_data.FieldMetaData("comment", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnDesc.class, metaDataMap); - } - - public TColumnDesc() { - } - - public TColumnDesc( - String columnName, - TTypeDesc typeDesc, - int position) - { - this(); - this.columnName = columnName; - this.typeDesc = typeDesc; - this.position = position; - setPositionIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TColumnDesc(TColumnDesc other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetColumnName()) { - this.columnName = other.columnName; - } - if (other.isSetTypeDesc()) { - this.typeDesc = new TTypeDesc(other.typeDesc); - } - this.position = other.position; - if (other.isSetComment()) { - this.comment = other.comment; - } - } - - public TColumnDesc deepCopy() { - return new TColumnDesc(this); - } - - @Override - public void clear() { - this.columnName = null; - this.typeDesc = null; - setPositionIsSet(false); - this.position = 0; - this.comment = null; - } - - public String getColumnName() { - return this.columnName; - } - - public void setColumnName(String columnName) { - this.columnName = columnName; - } - - public void unsetColumnName() { - this.columnName = null; - } - - /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ - public boolean isSetColumnName() { - return this.columnName != null; - } - - public void setColumnNameIsSet(boolean value) { - if (!value) { - this.columnName = null; - } - } - - public TTypeDesc getTypeDesc() { - return this.typeDesc; - } - - public void setTypeDesc(TTypeDesc typeDesc) { - this.typeDesc = typeDesc; - } - - public void unsetTypeDesc() { - this.typeDesc = null; - } - - /** Returns true if field typeDesc is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeDesc() { - return this.typeDesc != null; - } - - public void setTypeDescIsSet(boolean value) { - if (!value) { - this.typeDesc = null; - } - } - - public int getPosition() { - return this.position; - } - - public void setPosition(int position) { - this.position = position; - setPositionIsSet(true); - } - - public void unsetPosition() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __POSITION_ISSET_ID); - } - - /** Returns true if field position is set (has been assigned a value) and false otherwise */ - public boolean isSetPosition() { - return EncodingUtils.testBit(__isset_bitfield, __POSITION_ISSET_ID); - } - - public void setPositionIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __POSITION_ISSET_ID, value); - } - - public String getComment() { - return this.comment; - } - - public void setComment(String comment) { - this.comment = comment; - } - - public void unsetComment() { - this.comment = null; - } - - /** Returns true if field comment is set (has been assigned a value) and false otherwise */ - public boolean isSetComment() { - return this.comment != null; - } - - public void setCommentIsSet(boolean value) { - if (!value) { - this.comment = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COLUMN_NAME: - if (value == null) { - unsetColumnName(); - } else { - setColumnName((String)value); - } - break; - - case TYPE_DESC: - if (value == null) { - unsetTypeDesc(); - } else { - setTypeDesc((TTypeDesc)value); - } - break; - - case POSITION: - if (value == null) { - unsetPosition(); - } else { - setPosition((Integer)value); - } - break; - - case COMMENT: - if (value == null) { - unsetComment(); - } else { - setComment((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COLUMN_NAME: - return getColumnName(); - - case TYPE_DESC: - return getTypeDesc(); - - case POSITION: - return Integer.valueOf(getPosition()); - - case COMMENT: - return getComment(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COLUMN_NAME: - return isSetColumnName(); - case TYPE_DESC: - return isSetTypeDesc(); - case POSITION: - return isSetPosition(); - case COMMENT: - return isSetComment(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TColumnDesc) - return this.equals((TColumnDesc)that); - return false; - } - - public boolean equals(TColumnDesc that) { - if (that == null) - return false; - - boolean this_present_columnName = true && this.isSetColumnName(); - boolean that_present_columnName = true && that.isSetColumnName(); - if (this_present_columnName || that_present_columnName) { - if (!(this_present_columnName && that_present_columnName)) - return false; - if (!this.columnName.equals(that.columnName)) - return false; - } - - boolean this_present_typeDesc = true && this.isSetTypeDesc(); - boolean that_present_typeDesc = true && that.isSetTypeDesc(); - if (this_present_typeDesc || that_present_typeDesc) { - if (!(this_present_typeDesc && that_present_typeDesc)) - return false; - if (!this.typeDesc.equals(that.typeDesc)) - return false; - } - - boolean this_present_position = true; - boolean that_present_position = true; - if (this_present_position || that_present_position) { - if (!(this_present_position && that_present_position)) - return false; - if (this.position != that.position) - return false; - } - - boolean this_present_comment = true && this.isSetComment(); - boolean that_present_comment = true && that.isSetComment(); - if (this_present_comment || that_present_comment) { - if (!(this_present_comment && that_present_comment)) - return false; - if (!this.comment.equals(that.comment)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_columnName = true && (isSetColumnName()); - builder.append(present_columnName); - if (present_columnName) - builder.append(columnName); - - boolean present_typeDesc = true && (isSetTypeDesc()); - builder.append(present_typeDesc); - if (present_typeDesc) - builder.append(typeDesc); - - boolean present_position = true; - builder.append(present_position); - if (present_position) - builder.append(position); - - boolean present_comment = true && (isSetComment()); - builder.append(present_comment); - if (present_comment) - builder.append(comment); - - return builder.toHashCode(); - } - - public int compareTo(TColumnDesc other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TColumnDesc typedOther = (TColumnDesc)other; - - lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumnName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTypeDesc()).compareTo(typedOther.isSetTypeDesc()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeDesc()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeDesc, typedOther.typeDesc); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPosition()).compareTo(typedOther.isSetPosition()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPosition()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.position, typedOther.position); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetComment()).compareTo(typedOther.isSetComment()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetComment()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comment, typedOther.comment); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TColumnDesc("); - boolean first = true; - - sb.append("columnName:"); - if (this.columnName == null) { - sb.append("null"); - } else { - sb.append(this.columnName); - } - first = false; - if (!first) sb.append(", "); - sb.append("typeDesc:"); - if (this.typeDesc == null) { - sb.append("null"); - } else { - sb.append(this.typeDesc); - } - first = false; - if (!first) sb.append(", "); - sb.append("position:"); - sb.append(this.position); - first = false; - if (isSetComment()) { - if (!first) sb.append(", "); - sb.append("comment:"); - if (this.comment == null) { - sb.append("null"); - } else { - sb.append(this.comment); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColumnName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'columnName' is unset! Struct:" + toString()); - } - - if (!isSetTypeDesc()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeDesc' is unset! Struct:" + toString()); - } - - if (!isSetPosition()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'position' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (typeDesc != null) { - typeDesc.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TColumnDescStandardSchemeFactory implements SchemeFactory { - public TColumnDescStandardScheme getScheme() { - return new TColumnDescStandardScheme(); - } - } - - private static class TColumnDescStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnDesc struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COLUMN_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // TYPE_DESC - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.typeDesc = new TTypeDesc(); - struct.typeDesc.read(iprot); - struct.setTypeDescIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // POSITION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.position = iprot.readI32(); - struct.setPositionIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // COMMENT - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.comment = iprot.readString(); - struct.setCommentIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnDesc struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.columnName != null) { - oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); - oprot.writeString(struct.columnName); - oprot.writeFieldEnd(); - } - if (struct.typeDesc != null) { - oprot.writeFieldBegin(TYPE_DESC_FIELD_DESC); - struct.typeDesc.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(POSITION_FIELD_DESC); - oprot.writeI32(struct.position); - oprot.writeFieldEnd(); - if (struct.comment != null) { - if (struct.isSetComment()) { - oprot.writeFieldBegin(COMMENT_FIELD_DESC); - oprot.writeString(struct.comment); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TColumnDescTupleSchemeFactory implements SchemeFactory { - public TColumnDescTupleScheme getScheme() { - return new TColumnDescTupleScheme(); - } - } - - private static class TColumnDescTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeString(struct.columnName); - struct.typeDesc.write(oprot); - oprot.writeI32(struct.position); - BitSet optionals = new BitSet(); - if (struct.isSetComment()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetComment()) { - oprot.writeString(struct.comment); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - struct.typeDesc = new TTypeDesc(); - struct.typeDesc.read(iprot); - struct.setTypeDescIsSet(true); - struct.position = iprot.readI32(); - struct.setPositionIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.comment = iprot.readString(); - struct.setCommentIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java deleted file mode 100644 index 8504c6d608d42..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java +++ /dev/null @@ -1,671 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TColumnValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); - private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); - private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - BOOL_VAL((short)1, "boolVal"), - BYTE_VAL((short)2, "byteVal"), - I16_VAL((short)3, "i16Val"), - I32_VAL((short)4, "i32Val"), - I64_VAL((short)5, "i64Val"), - DOUBLE_VAL((short)6, "doubleVal"), - STRING_VAL((short)7, "stringVal"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOOL_VAL - return BOOL_VAL; - case 2: // BYTE_VAL - return BYTE_VAL; - case 3: // I16_VAL - return I16_VAL; - case 4: // I32_VAL - return I32_VAL; - case 5: // I64_VAL - return I64_VAL; - case 6: // DOUBLE_VAL - return DOUBLE_VAL; - case 7: // STRING_VAL - return STRING_VAL; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolValue.class))); - tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteValue.class))); - tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Value.class))); - tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Value.class))); - tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Value.class))); - tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleValue.class))); - tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringValue.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap); - } - - public TColumnValue() { - super(); - } - - public TColumnValue(TColumnValue._Fields setField, Object value) { - super(setField, value); - } - - public TColumnValue(TColumnValue other) { - super(other); - } - public TColumnValue deepCopy() { - return new TColumnValue(this); - } - - public static TColumnValue boolVal(TBoolValue value) { - TColumnValue x = new TColumnValue(); - x.setBoolVal(value); - return x; - } - - public static TColumnValue byteVal(TByteValue value) { - TColumnValue x = new TColumnValue(); - x.setByteVal(value); - return x; - } - - public static TColumnValue i16Val(TI16Value value) { - TColumnValue x = new TColumnValue(); - x.setI16Val(value); - return x; - } - - public static TColumnValue i32Val(TI32Value value) { - TColumnValue x = new TColumnValue(); - x.setI32Val(value); - return x; - } - - public static TColumnValue i64Val(TI64Value value) { - TColumnValue x = new TColumnValue(); - x.setI64Val(value); - return x; - } - - public static TColumnValue doubleVal(TDoubleValue value) { - TColumnValue x = new TColumnValue(); - x.setDoubleVal(value); - return x; - } - - public static TColumnValue stringVal(TStringValue value) { - TColumnValue x = new TColumnValue(); - x.setStringVal(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case BOOL_VAL: - if (value instanceof TBoolValue) { - break; - } - throw new ClassCastException("Was expecting value of type TBoolValue for field 'boolVal', but got " + value.getClass().getSimpleName()); - case BYTE_VAL: - if (value instanceof TByteValue) { - break; - } - throw new ClassCastException("Was expecting value of type TByteValue for field 'byteVal', but got " + value.getClass().getSimpleName()); - case I16_VAL: - if (value instanceof TI16Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI16Value for field 'i16Val', but got " + value.getClass().getSimpleName()); - case I32_VAL: - if (value instanceof TI32Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI32Value for field 'i32Val', but got " + value.getClass().getSimpleName()); - case I64_VAL: - if (value instanceof TI64Value) { - break; - } - throw new ClassCastException("Was expecting value of type TI64Value for field 'i64Val', but got " + value.getClass().getSimpleName()); - case DOUBLE_VAL: - if (value instanceof TDoubleValue) { - break; - } - throw new ClassCastException("Was expecting value of type TDoubleValue for field 'doubleVal', but got " + value.getClass().getSimpleName()); - case STRING_VAL: - if (value instanceof TStringValue) { - break; - } - throw new ClassCastException("Was expecting value of type TStringValue for field 'stringVal', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - if (field.type == BOOL_VAL_FIELD_DESC.type) { - TBoolValue boolVal; - boolVal = new TBoolValue(); - boolVal.read(iprot); - return boolVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BYTE_VAL: - if (field.type == BYTE_VAL_FIELD_DESC.type) { - TByteValue byteVal; - byteVal = new TByteValue(); - byteVal.read(iprot); - return byteVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I16_VAL: - if (field.type == I16_VAL_FIELD_DESC.type) { - TI16Value i16Val; - i16Val = new TI16Value(); - i16Val.read(iprot); - return i16Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I32_VAL: - if (field.type == I32_VAL_FIELD_DESC.type) { - TI32Value i32Val; - i32Val = new TI32Value(); - i32Val.read(iprot); - return i32Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case I64_VAL: - if (field.type == I64_VAL_FIELD_DESC.type) { - TI64Value i64Val; - i64Val = new TI64Value(); - i64Val.read(iprot); - return i64Val; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DOUBLE_VAL: - if (field.type == DOUBLE_VAL_FIELD_DESC.type) { - TDoubleValue doubleVal; - doubleVal = new TDoubleValue(); - doubleVal.read(iprot); - return doubleVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VAL: - if (field.type == STRING_VAL_FIELD_DESC.type) { - TStringValue stringVal; - stringVal = new TStringValue(); - stringVal.read(iprot); - return stringVal; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolValue boolVal = (TBoolValue)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteValue byteVal = (TByteValue)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Value i16Val = (TI16Value)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Value i32Val = (TI32Value)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Value i64Val = (TI64Value)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleValue doubleVal = (TDoubleValue)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringValue stringVal = (TStringValue)value_; - stringVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case BOOL_VAL: - TBoolValue boolVal; - boolVal = new TBoolValue(); - boolVal.read(iprot); - return boolVal; - case BYTE_VAL: - TByteValue byteVal; - byteVal = new TByteValue(); - byteVal.read(iprot); - return byteVal; - case I16_VAL: - TI16Value i16Val; - i16Val = new TI16Value(); - i16Val.read(iprot); - return i16Val; - case I32_VAL: - TI32Value i32Val; - i32Val = new TI32Value(); - i32Val.read(iprot); - return i32Val; - case I64_VAL: - TI64Value i64Val; - i64Val = new TI64Value(); - i64Val.read(iprot); - return i64Val; - case DOUBLE_VAL: - TDoubleValue doubleVal; - doubleVal = new TDoubleValue(); - doubleVal.read(iprot); - return doubleVal; - case STRING_VAL: - TStringValue stringVal; - stringVal = new TStringValue(); - stringVal.read(iprot); - return stringVal; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case BOOL_VAL: - TBoolValue boolVal = (TBoolValue)value_; - boolVal.write(oprot); - return; - case BYTE_VAL: - TByteValue byteVal = (TByteValue)value_; - byteVal.write(oprot); - return; - case I16_VAL: - TI16Value i16Val = (TI16Value)value_; - i16Val.write(oprot); - return; - case I32_VAL: - TI32Value i32Val = (TI32Value)value_; - i32Val.write(oprot); - return; - case I64_VAL: - TI64Value i64Val = (TI64Value)value_; - i64Val.write(oprot); - return; - case DOUBLE_VAL: - TDoubleValue doubleVal = (TDoubleValue)value_; - doubleVal.write(oprot); - return; - case STRING_VAL: - TStringValue stringVal = (TStringValue)value_; - stringVal.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case BOOL_VAL: - return BOOL_VAL_FIELD_DESC; - case BYTE_VAL: - return BYTE_VAL_FIELD_DESC; - case I16_VAL: - return I16_VAL_FIELD_DESC; - case I32_VAL: - return I32_VAL_FIELD_DESC; - case I64_VAL: - return I64_VAL_FIELD_DESC; - case DOUBLE_VAL: - return DOUBLE_VAL_FIELD_DESC; - case STRING_VAL: - return STRING_VAL_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TBoolValue getBoolVal() { - if (getSetField() == _Fields.BOOL_VAL) { - return (TBoolValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBoolVal(TBoolValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BOOL_VAL; - value_ = value; - } - - public TByteValue getByteVal() { - if (getSetField() == _Fields.BYTE_VAL) { - return (TByteValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setByteVal(TByteValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BYTE_VAL; - value_ = value; - } - - public TI16Value getI16Val() { - if (getSetField() == _Fields.I16_VAL) { - return (TI16Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI16Val(TI16Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I16_VAL; - value_ = value; - } - - public TI32Value getI32Val() { - if (getSetField() == _Fields.I32_VAL) { - return (TI32Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Val(TI32Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I32_VAL; - value_ = value; - } - - public TI64Value getI64Val() { - if (getSetField() == _Fields.I64_VAL) { - return (TI64Value)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI64Val(TI64Value value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.I64_VAL; - value_ = value; - } - - public TDoubleValue getDoubleVal() { - if (getSetField() == _Fields.DOUBLE_VAL) { - return (TDoubleValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setDoubleVal(TDoubleValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.DOUBLE_VAL; - value_ = value; - } - - public TStringValue getStringVal() { - if (getSetField() == _Fields.STRING_VAL) { - return (TStringValue)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringVal(TStringValue value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VAL; - value_ = value; - } - - public boolean isSetBoolVal() { - return setField_ == _Fields.BOOL_VAL; - } - - - public boolean isSetByteVal() { - return setField_ == _Fields.BYTE_VAL; - } - - - public boolean isSetI16Val() { - return setField_ == _Fields.I16_VAL; - } - - - public boolean isSetI32Val() { - return setField_ == _Fields.I32_VAL; - } - - - public boolean isSetI64Val() { - return setField_ == _Fields.I64_VAL; - } - - - public boolean isSetDoubleVal() { - return setField_ == _Fields.DOUBLE_VAL; - } - - - public boolean isSetStringVal() { - return setField_ == _Fields.STRING_VAL; - } - - - public boolean equals(Object other) { - if (other instanceof TColumnValue) { - return equals((TColumnValue)other); - } else { - return false; - } - } - - public boolean equals(TColumnValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TColumnValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java deleted file mode 100644 index 4fc54544c1bea..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TDoubleColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TDoubleColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TDoubleColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleColumn.class, metaDataMap); - } - - public TDoubleColumn() { - } - - public TDoubleColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TDoubleColumn(TDoubleColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Double other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TDoubleColumn deepCopy() { - return new TDoubleColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(double elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TDoubleColumn) - return this.equals((TDoubleColumn)that); - return false; - } - - public boolean equals(TDoubleColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TDoubleColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TDoubleColumn typedOther = (TDoubleColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TDoubleColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TDoubleColumnStandardSchemeFactory implements SchemeFactory { - public TDoubleColumnStandardScheme getScheme() { - return new TDoubleColumnStandardScheme(); - } - } - - private static class TDoubleColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list94 = iprot.readListBegin(); - struct.values = new ArrayList(_list94.size); - for (int _i95 = 0; _i95 < _list94.size; ++_i95) - { - double _elem96; // optional - _elem96 = iprot.readDouble(); - struct.values.add(_elem96); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, struct.values.size())); - for (double _iter97 : struct.values) - { - oprot.writeDouble(_iter97); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TDoubleColumnTupleSchemeFactory implements SchemeFactory { - public TDoubleColumnTupleScheme getScheme() { - return new TDoubleColumnTupleScheme(); - } - } - - private static class TDoubleColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (double _iter98 : struct.values) - { - oprot.writeDouble(_iter98); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); - struct.values = new ArrayList(_list99.size); - for (int _i100 = 0; _i100 < _list99.size; ++_i100) - { - double _elem101; // optional - _elem101 = iprot.readDouble(); - struct.values.add(_elem101); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java deleted file mode 100644 index d21573633ef51..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TDoubleValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.DOUBLE, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TDoubleValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TDoubleValueTupleSchemeFactory()); - } - - private double value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleValue.class, metaDataMap); - } - - public TDoubleValue() { - } - - /** - * Performs a deep copy on other. - */ - public TDoubleValue(TDoubleValue other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TDoubleValue deepCopy() { - return new TDoubleValue(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0.0; - } - - public double getValue() { - return this.value; - } - - public void setValue(double value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Double)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Double.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TDoubleValue) - return this.equals((TDoubleValue)that); - return false; - } - - public boolean equals(TDoubleValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TDoubleValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TDoubleValue typedOther = (TDoubleValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TDoubleValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TDoubleValueStandardSchemeFactory implements SchemeFactory { - public TDoubleValueStandardScheme getScheme() { - return new TDoubleValueStandardScheme(); - } - } - - private static class TDoubleValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { - struct.value = iprot.readDouble(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeDouble(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TDoubleValueTupleSchemeFactory implements SchemeFactory { - public TDoubleValueTupleScheme getScheme() { - return new TDoubleValueTupleScheme(); - } - } - - private static class TDoubleValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeDouble(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readDouble(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java deleted file mode 100644 index 4f157ad5a6450..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java +++ /dev/null @@ -1,769 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TExecuteStatementReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField STATEMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("statement", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField CONF_OVERLAY_FIELD_DESC = new org.apache.thrift.protocol.TField("confOverlay", org.apache.thrift.protocol.TType.MAP, (short)3); - private static final org.apache.thrift.protocol.TField RUN_ASYNC_FIELD_DESC = new org.apache.thrift.protocol.TField("runAsync", org.apache.thrift.protocol.TType.BOOL, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TExecuteStatementReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TExecuteStatementReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String statement; // required - private Map confOverlay; // optional - private boolean runAsync; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - STATEMENT((short)2, "statement"), - CONF_OVERLAY((short)3, "confOverlay"), - RUN_ASYNC((short)4, "runAsync"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // STATEMENT - return STATEMENT; - case 3: // CONF_OVERLAY - return CONF_OVERLAY; - case 4: // RUN_ASYNC - return RUN_ASYNC; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __RUNASYNC_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.STATEMENT, new org.apache.thrift.meta_data.FieldMetaData("statement", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.CONF_OVERLAY, new org.apache.thrift.meta_data.FieldMetaData("confOverlay", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.RUN_ASYNC, new org.apache.thrift.meta_data.FieldMetaData("runAsync", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementReq.class, metaDataMap); - } - - public TExecuteStatementReq() { - this.runAsync = false; - - } - - public TExecuteStatementReq( - TSessionHandle sessionHandle, - String statement) - { - this(); - this.sessionHandle = sessionHandle; - this.statement = statement; - } - - /** - * Performs a deep copy on other. - */ - public TExecuteStatementReq(TExecuteStatementReq other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetStatement()) { - this.statement = other.statement; - } - if (other.isSetConfOverlay()) { - Map __this__confOverlay = new HashMap(); - for (Map.Entry other_element : other.confOverlay.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__confOverlay_copy_key = other_element_key; - - String __this__confOverlay_copy_value = other_element_value; - - __this__confOverlay.put(__this__confOverlay_copy_key, __this__confOverlay_copy_value); - } - this.confOverlay = __this__confOverlay; - } - this.runAsync = other.runAsync; - } - - public TExecuteStatementReq deepCopy() { - return new TExecuteStatementReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.statement = null; - this.confOverlay = null; - this.runAsync = false; - - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getStatement() { - return this.statement; - } - - public void setStatement(String statement) { - this.statement = statement; - } - - public void unsetStatement() { - this.statement = null; - } - - /** Returns true if field statement is set (has been assigned a value) and false otherwise */ - public boolean isSetStatement() { - return this.statement != null; - } - - public void setStatementIsSet(boolean value) { - if (!value) { - this.statement = null; - } - } - - public int getConfOverlaySize() { - return (this.confOverlay == null) ? 0 : this.confOverlay.size(); - } - - public void putToConfOverlay(String key, String val) { - if (this.confOverlay == null) { - this.confOverlay = new HashMap(); - } - this.confOverlay.put(key, val); - } - - public Map getConfOverlay() { - return this.confOverlay; - } - - public void setConfOverlay(Map confOverlay) { - this.confOverlay = confOverlay; - } - - public void unsetConfOverlay() { - this.confOverlay = null; - } - - /** Returns true if field confOverlay is set (has been assigned a value) and false otherwise */ - public boolean isSetConfOverlay() { - return this.confOverlay != null; - } - - public void setConfOverlayIsSet(boolean value) { - if (!value) { - this.confOverlay = null; - } - } - - public boolean isRunAsync() { - return this.runAsync; - } - - public void setRunAsync(boolean runAsync) { - this.runAsync = runAsync; - setRunAsyncIsSet(true); - } - - public void unsetRunAsync() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RUNASYNC_ISSET_ID); - } - - /** Returns true if field runAsync is set (has been assigned a value) and false otherwise */ - public boolean isSetRunAsync() { - return EncodingUtils.testBit(__isset_bitfield, __RUNASYNC_ISSET_ID); - } - - public void setRunAsyncIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RUNASYNC_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case STATEMENT: - if (value == null) { - unsetStatement(); - } else { - setStatement((String)value); - } - break; - - case CONF_OVERLAY: - if (value == null) { - unsetConfOverlay(); - } else { - setConfOverlay((Map)value); - } - break; - - case RUN_ASYNC: - if (value == null) { - unsetRunAsync(); - } else { - setRunAsync((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case STATEMENT: - return getStatement(); - - case CONF_OVERLAY: - return getConfOverlay(); - - case RUN_ASYNC: - return Boolean.valueOf(isRunAsync()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case STATEMENT: - return isSetStatement(); - case CONF_OVERLAY: - return isSetConfOverlay(); - case RUN_ASYNC: - return isSetRunAsync(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TExecuteStatementReq) - return this.equals((TExecuteStatementReq)that); - return false; - } - - public boolean equals(TExecuteStatementReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_statement = true && this.isSetStatement(); - boolean that_present_statement = true && that.isSetStatement(); - if (this_present_statement || that_present_statement) { - if (!(this_present_statement && that_present_statement)) - return false; - if (!this.statement.equals(that.statement)) - return false; - } - - boolean this_present_confOverlay = true && this.isSetConfOverlay(); - boolean that_present_confOverlay = true && that.isSetConfOverlay(); - if (this_present_confOverlay || that_present_confOverlay) { - if (!(this_present_confOverlay && that_present_confOverlay)) - return false; - if (!this.confOverlay.equals(that.confOverlay)) - return false; - } - - boolean this_present_runAsync = true && this.isSetRunAsync(); - boolean that_present_runAsync = true && that.isSetRunAsync(); - if (this_present_runAsync || that_present_runAsync) { - if (!(this_present_runAsync && that_present_runAsync)) - return false; - if (this.runAsync != that.runAsync) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_statement = true && (isSetStatement()); - builder.append(present_statement); - if (present_statement) - builder.append(statement); - - boolean present_confOverlay = true && (isSetConfOverlay()); - builder.append(present_confOverlay); - if (present_confOverlay) - builder.append(confOverlay); - - boolean present_runAsync = true && (isSetRunAsync()); - builder.append(present_runAsync); - if (present_runAsync) - builder.append(runAsync); - - return builder.toHashCode(); - } - - public int compareTo(TExecuteStatementReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TExecuteStatementReq typedOther = (TExecuteStatementReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetStatement()).compareTo(typedOther.isSetStatement()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatement()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statement, typedOther.statement); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfOverlay()).compareTo(typedOther.isSetConfOverlay()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfOverlay()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.confOverlay, typedOther.confOverlay); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRunAsync()).compareTo(typedOther.isSetRunAsync()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRunAsync()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runAsync, typedOther.runAsync); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TExecuteStatementReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("statement:"); - if (this.statement == null) { - sb.append("null"); - } else { - sb.append(this.statement); - } - first = false; - if (isSetConfOverlay()) { - if (!first) sb.append(", "); - sb.append("confOverlay:"); - if (this.confOverlay == null) { - sb.append("null"); - } else { - sb.append(this.confOverlay); - } - first = false; - } - if (isSetRunAsync()) { - if (!first) sb.append(", "); - sb.append("runAsync:"); - sb.append(this.runAsync); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetStatement()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'statement' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TExecuteStatementReqStandardSchemeFactory implements SchemeFactory { - public TExecuteStatementReqStandardScheme getScheme() { - return new TExecuteStatementReqStandardScheme(); - } - } - - private static class TExecuteStatementReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // STATEMENT - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.statement = iprot.readString(); - struct.setStatementIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // CONF_OVERLAY - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map162 = iprot.readMapBegin(); - struct.confOverlay = new HashMap(2*_map162.size); - for (int _i163 = 0; _i163 < _map162.size; ++_i163) - { - String _key164; // required - String _val165; // required - _key164 = iprot.readString(); - _val165 = iprot.readString(); - struct.confOverlay.put(_key164, _val165); - } - iprot.readMapEnd(); - } - struct.setConfOverlayIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // RUN_ASYNC - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.runAsync = iprot.readBool(); - struct.setRunAsyncIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.statement != null) { - oprot.writeFieldBegin(STATEMENT_FIELD_DESC); - oprot.writeString(struct.statement); - oprot.writeFieldEnd(); - } - if (struct.confOverlay != null) { - if (struct.isSetConfOverlay()) { - oprot.writeFieldBegin(CONF_OVERLAY_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.confOverlay.size())); - for (Map.Entry _iter166 : struct.confOverlay.entrySet()) - { - oprot.writeString(_iter166.getKey()); - oprot.writeString(_iter166.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - if (struct.isSetRunAsync()) { - oprot.writeFieldBegin(RUN_ASYNC_FIELD_DESC); - oprot.writeBool(struct.runAsync); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TExecuteStatementReqTupleSchemeFactory implements SchemeFactory { - public TExecuteStatementReqTupleScheme getScheme() { - return new TExecuteStatementReqTupleScheme(); - } - } - - private static class TExecuteStatementReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.statement); - BitSet optionals = new BitSet(); - if (struct.isSetConfOverlay()) { - optionals.set(0); - } - if (struct.isSetRunAsync()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetConfOverlay()) { - { - oprot.writeI32(struct.confOverlay.size()); - for (Map.Entry _iter167 : struct.confOverlay.entrySet()) - { - oprot.writeString(_iter167.getKey()); - oprot.writeString(_iter167.getValue()); - } - } - } - if (struct.isSetRunAsync()) { - oprot.writeBool(struct.runAsync); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.statement = iprot.readString(); - struct.setStatementIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.confOverlay = new HashMap(2*_map168.size); - for (int _i169 = 0; _i169 < _map168.size; ++_i169) - { - String _key170; // required - String _val171; // required - _key170 = iprot.readString(); - _val171 = iprot.readString(); - struct.confOverlay.put(_key170, _val171); - } - } - struct.setConfOverlayIsSet(true); - } - if (incoming.get(1)) { - struct.runAsync = iprot.readBool(); - struct.setRunAsyncIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java deleted file mode 100644 index fdde51e70f783..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TExecuteStatementResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TExecuteStatementRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TExecuteStatementRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementResp.class, metaDataMap); - } - - public TExecuteStatementResp() { - } - - public TExecuteStatementResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TExecuteStatementResp(TExecuteStatementResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TExecuteStatementResp deepCopy() { - return new TExecuteStatementResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TExecuteStatementResp) - return this.equals((TExecuteStatementResp)that); - return false; - } - - public boolean equals(TExecuteStatementResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TExecuteStatementResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TExecuteStatementResp typedOther = (TExecuteStatementResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TExecuteStatementResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TExecuteStatementRespStandardSchemeFactory implements SchemeFactory { - public TExecuteStatementRespStandardScheme getScheme() { - return new TExecuteStatementRespStandardScheme(); - } - } - - private static class TExecuteStatementRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TExecuteStatementRespTupleSchemeFactory implements SchemeFactory { - public TExecuteStatementRespTupleScheme getScheme() { - return new TExecuteStatementRespTupleScheme(); - } - } - - private static class TExecuteStatementRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java deleted file mode 100644 index b2a22effd91af..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TFetchOrientation implements org.apache.thrift.TEnum { - FETCH_NEXT(0), - FETCH_PRIOR(1), - FETCH_RELATIVE(2), - FETCH_ABSOLUTE(3), - FETCH_FIRST(4), - FETCH_LAST(5); - - private final int value; - - private TFetchOrientation(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TFetchOrientation findByValue(int value) { - switch (value) { - case 0: - return FETCH_NEXT; - case 1: - return FETCH_PRIOR; - case 2: - return FETCH_RELATIVE; - case 3: - return FETCH_ABSOLUTE; - case 4: - return FETCH_FIRST; - case 5: - return FETCH_LAST; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java deleted file mode 100644 index 068711fc44440..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java +++ /dev/null @@ -1,710 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TFetchResultsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField ORIENTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("orientation", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField MAX_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxRows", org.apache.thrift.protocol.TType.I64, (short)3); - private static final org.apache.thrift.protocol.TField FETCH_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("fetchType", org.apache.thrift.protocol.TType.I16, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TFetchResultsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TFetchResultsReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - private TFetchOrientation orientation; // required - private long maxRows; // required - private short fetchType; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"), - /** - * - * @see TFetchOrientation - */ - ORIENTATION((short)2, "orientation"), - MAX_ROWS((short)3, "maxRows"), - FETCH_TYPE((short)4, "fetchType"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - case 2: // ORIENTATION - return ORIENTATION; - case 3: // MAX_ROWS - return MAX_ROWS; - case 4: // FETCH_TYPE - return FETCH_TYPE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __MAXROWS_ISSET_ID = 0; - private static final int __FETCHTYPE_ISSET_ID = 1; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.FETCH_TYPE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - tmpMap.put(_Fields.ORIENTATION, new org.apache.thrift.meta_data.FieldMetaData("orientation", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFetchOrientation.class))); - tmpMap.put(_Fields.MAX_ROWS, new org.apache.thrift.meta_data.FieldMetaData("maxRows", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.FETCH_TYPE, new org.apache.thrift.meta_data.FieldMetaData("fetchType", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsReq.class, metaDataMap); - } - - public TFetchResultsReq() { - this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; - - this.fetchType = (short)0; - - } - - public TFetchResultsReq( - TOperationHandle operationHandle, - TFetchOrientation orientation, - long maxRows) - { - this(); - this.operationHandle = operationHandle; - this.orientation = orientation; - this.maxRows = maxRows; - setMaxRowsIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TFetchResultsReq(TFetchResultsReq other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - if (other.isSetOrientation()) { - this.orientation = other.orientation; - } - this.maxRows = other.maxRows; - this.fetchType = other.fetchType; - } - - public TFetchResultsReq deepCopy() { - return new TFetchResultsReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; - - setMaxRowsIsSet(false); - this.maxRows = 0; - this.fetchType = (short)0; - - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - /** - * - * @see TFetchOrientation - */ - public TFetchOrientation getOrientation() { - return this.orientation; - } - - /** - * - * @see TFetchOrientation - */ - public void setOrientation(TFetchOrientation orientation) { - this.orientation = orientation; - } - - public void unsetOrientation() { - this.orientation = null; - } - - /** Returns true if field orientation is set (has been assigned a value) and false otherwise */ - public boolean isSetOrientation() { - return this.orientation != null; - } - - public void setOrientationIsSet(boolean value) { - if (!value) { - this.orientation = null; - } - } - - public long getMaxRows() { - return this.maxRows; - } - - public void setMaxRows(long maxRows) { - this.maxRows = maxRows; - setMaxRowsIsSet(true); - } - - public void unsetMaxRows() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXROWS_ISSET_ID); - } - - /** Returns true if field maxRows is set (has been assigned a value) and false otherwise */ - public boolean isSetMaxRows() { - return EncodingUtils.testBit(__isset_bitfield, __MAXROWS_ISSET_ID); - } - - public void setMaxRowsIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXROWS_ISSET_ID, value); - } - - public short getFetchType() { - return this.fetchType; - } - - public void setFetchType(short fetchType) { - this.fetchType = fetchType; - setFetchTypeIsSet(true); - } - - public void unsetFetchType() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); - } - - /** Returns true if field fetchType is set (has been assigned a value) and false otherwise */ - public boolean isSetFetchType() { - return EncodingUtils.testBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); - } - - public void setFetchTypeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FETCHTYPE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - case ORIENTATION: - if (value == null) { - unsetOrientation(); - } else { - setOrientation((TFetchOrientation)value); - } - break; - - case MAX_ROWS: - if (value == null) { - unsetMaxRows(); - } else { - setMaxRows((Long)value); - } - break; - - case FETCH_TYPE: - if (value == null) { - unsetFetchType(); - } else { - setFetchType((Short)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - case ORIENTATION: - return getOrientation(); - - case MAX_ROWS: - return Long.valueOf(getMaxRows()); - - case FETCH_TYPE: - return Short.valueOf(getFetchType()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - case ORIENTATION: - return isSetOrientation(); - case MAX_ROWS: - return isSetMaxRows(); - case FETCH_TYPE: - return isSetFetchType(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TFetchResultsReq) - return this.equals((TFetchResultsReq)that); - return false; - } - - public boolean equals(TFetchResultsReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - boolean this_present_orientation = true && this.isSetOrientation(); - boolean that_present_orientation = true && that.isSetOrientation(); - if (this_present_orientation || that_present_orientation) { - if (!(this_present_orientation && that_present_orientation)) - return false; - if (!this.orientation.equals(that.orientation)) - return false; - } - - boolean this_present_maxRows = true; - boolean that_present_maxRows = true; - if (this_present_maxRows || that_present_maxRows) { - if (!(this_present_maxRows && that_present_maxRows)) - return false; - if (this.maxRows != that.maxRows) - return false; - } - - boolean this_present_fetchType = true && this.isSetFetchType(); - boolean that_present_fetchType = true && that.isSetFetchType(); - if (this_present_fetchType || that_present_fetchType) { - if (!(this_present_fetchType && that_present_fetchType)) - return false; - if (this.fetchType != that.fetchType) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - boolean present_orientation = true && (isSetOrientation()); - builder.append(present_orientation); - if (present_orientation) - builder.append(orientation.getValue()); - - boolean present_maxRows = true; - builder.append(present_maxRows); - if (present_maxRows) - builder.append(maxRows); - - boolean present_fetchType = true && (isSetFetchType()); - builder.append(present_fetchType); - if (present_fetchType) - builder.append(fetchType); - - return builder.toHashCode(); - } - - public int compareTo(TFetchResultsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TFetchResultsReq typedOther = (TFetchResultsReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOrientation()).compareTo(typedOther.isSetOrientation()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOrientation()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.orientation, typedOther.orientation); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetMaxRows()).compareTo(typedOther.isSetMaxRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetMaxRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxRows, typedOther.maxRows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetFetchType()).compareTo(typedOther.isSetFetchType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetFetchType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fetchType, typedOther.fetchType); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TFetchResultsReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("orientation:"); - if (this.orientation == null) { - sb.append("null"); - } else { - sb.append(this.orientation); - } - first = false; - if (!first) sb.append(", "); - sb.append("maxRows:"); - sb.append(this.maxRows); - first = false; - if (isSetFetchType()) { - if (!first) sb.append(", "); - sb.append("fetchType:"); - sb.append(this.fetchType); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - if (!isSetOrientation()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'orientation' is unset! Struct:" + toString()); - } - - if (!isSetMaxRows()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxRows' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TFetchResultsReqStandardSchemeFactory implements SchemeFactory { - public TFetchResultsReqStandardScheme getScheme() { - return new TFetchResultsReqStandardScheme(); - } - } - - private static class TFetchResultsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // ORIENTATION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); - struct.setOrientationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // MAX_ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.maxRows = iprot.readI64(); - struct.setMaxRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // FETCH_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I16) { - struct.fetchType = iprot.readI16(); - struct.setFetchTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.orientation != null) { - oprot.writeFieldBegin(ORIENTATION_FIELD_DESC); - oprot.writeI32(struct.orientation.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(MAX_ROWS_FIELD_DESC); - oprot.writeI64(struct.maxRows); - oprot.writeFieldEnd(); - if (struct.isSetFetchType()) { - oprot.writeFieldBegin(FETCH_TYPE_FIELD_DESC); - oprot.writeI16(struct.fetchType); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TFetchResultsReqTupleSchemeFactory implements SchemeFactory { - public TFetchResultsReqTupleScheme getScheme() { - return new TFetchResultsReqTupleScheme(); - } - } - - private static class TFetchResultsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - oprot.writeI32(struct.orientation.getValue()); - oprot.writeI64(struct.maxRows); - BitSet optionals = new BitSet(); - if (struct.isSetFetchType()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetFetchType()) { - oprot.writeI16(struct.fetchType); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); - struct.setOrientationIsSet(true); - struct.maxRows = iprot.readI64(); - struct.setMaxRowsIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.fetchType = iprot.readI16(); - struct.setFetchTypeIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java deleted file mode 100644 index 19991f1da3eb3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java +++ /dev/null @@ -1,608 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TFetchResultsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField HAS_MORE_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("hasMoreRows", org.apache.thrift.protocol.TType.BOOL, (short)2); - private static final org.apache.thrift.protocol.TField RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("results", org.apache.thrift.protocol.TType.STRUCT, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TFetchResultsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TFetchResultsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private boolean hasMoreRows; // optional - private TRowSet results; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - HAS_MORE_ROWS((short)2, "hasMoreRows"), - RESULTS((short)3, "results"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // HAS_MORE_ROWS - return HAS_MORE_ROWS; - case 3: // RESULTS - return RESULTS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __HASMOREROWS_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.HAS_MORE_ROWS,_Fields.RESULTS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.HAS_MORE_ROWS, new org.apache.thrift.meta_data.FieldMetaData("hasMoreRows", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - tmpMap.put(_Fields.RESULTS, new org.apache.thrift.meta_data.FieldMetaData("results", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowSet.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsResp.class, metaDataMap); - } - - public TFetchResultsResp() { - } - - public TFetchResultsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TFetchResultsResp(TFetchResultsResp other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - this.hasMoreRows = other.hasMoreRows; - if (other.isSetResults()) { - this.results = new TRowSet(other.results); - } - } - - public TFetchResultsResp deepCopy() { - return new TFetchResultsResp(this); - } - - @Override - public void clear() { - this.status = null; - setHasMoreRowsIsSet(false); - this.hasMoreRows = false; - this.results = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public boolean isHasMoreRows() { - return this.hasMoreRows; - } - - public void setHasMoreRows(boolean hasMoreRows) { - this.hasMoreRows = hasMoreRows; - setHasMoreRowsIsSet(true); - } - - public void unsetHasMoreRows() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); - } - - /** Returns true if field hasMoreRows is set (has been assigned a value) and false otherwise */ - public boolean isSetHasMoreRows() { - return EncodingUtils.testBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); - } - - public void setHasMoreRowsIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASMOREROWS_ISSET_ID, value); - } - - public TRowSet getResults() { - return this.results; - } - - public void setResults(TRowSet results) { - this.results = results; - } - - public void unsetResults() { - this.results = null; - } - - /** Returns true if field results is set (has been assigned a value) and false otherwise */ - public boolean isSetResults() { - return this.results != null; - } - - public void setResultsIsSet(boolean value) { - if (!value) { - this.results = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case HAS_MORE_ROWS: - if (value == null) { - unsetHasMoreRows(); - } else { - setHasMoreRows((Boolean)value); - } - break; - - case RESULTS: - if (value == null) { - unsetResults(); - } else { - setResults((TRowSet)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case HAS_MORE_ROWS: - return Boolean.valueOf(isHasMoreRows()); - - case RESULTS: - return getResults(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case HAS_MORE_ROWS: - return isSetHasMoreRows(); - case RESULTS: - return isSetResults(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TFetchResultsResp) - return this.equals((TFetchResultsResp)that); - return false; - } - - public boolean equals(TFetchResultsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_hasMoreRows = true && this.isSetHasMoreRows(); - boolean that_present_hasMoreRows = true && that.isSetHasMoreRows(); - if (this_present_hasMoreRows || that_present_hasMoreRows) { - if (!(this_present_hasMoreRows && that_present_hasMoreRows)) - return false; - if (this.hasMoreRows != that.hasMoreRows) - return false; - } - - boolean this_present_results = true && this.isSetResults(); - boolean that_present_results = true && that.isSetResults(); - if (this_present_results || that_present_results) { - if (!(this_present_results && that_present_results)) - return false; - if (!this.results.equals(that.results)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_hasMoreRows = true && (isSetHasMoreRows()); - builder.append(present_hasMoreRows); - if (present_hasMoreRows) - builder.append(hasMoreRows); - - boolean present_results = true && (isSetResults()); - builder.append(present_results); - if (present_results) - builder.append(results); - - return builder.toHashCode(); - } - - public int compareTo(TFetchResultsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TFetchResultsResp typedOther = (TFetchResultsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetHasMoreRows()).compareTo(typedOther.isSetHasMoreRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetHasMoreRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasMoreRows, typedOther.hasMoreRows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetResults()).compareTo(typedOther.isSetResults()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetResults()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.results, typedOther.results); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TFetchResultsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetHasMoreRows()) { - if (!first) sb.append(", "); - sb.append("hasMoreRows:"); - sb.append(this.hasMoreRows); - first = false; - } - if (isSetResults()) { - if (!first) sb.append(", "); - sb.append("results:"); - if (this.results == null) { - sb.append("null"); - } else { - sb.append(this.results); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (results != null) { - results.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TFetchResultsRespStandardSchemeFactory implements SchemeFactory { - public TFetchResultsRespStandardScheme getScheme() { - return new TFetchResultsRespStandardScheme(); - } - } - - private static class TFetchResultsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // HAS_MORE_ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.hasMoreRows = iprot.readBool(); - struct.setHasMoreRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // RESULTS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.results = new TRowSet(); - struct.results.read(iprot); - struct.setResultsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.isSetHasMoreRows()) { - oprot.writeFieldBegin(HAS_MORE_ROWS_FIELD_DESC); - oprot.writeBool(struct.hasMoreRows); - oprot.writeFieldEnd(); - } - if (struct.results != null) { - if (struct.isSetResults()) { - oprot.writeFieldBegin(RESULTS_FIELD_DESC); - struct.results.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TFetchResultsRespTupleSchemeFactory implements SchemeFactory { - public TFetchResultsRespTupleScheme getScheme() { - return new TFetchResultsRespTupleScheme(); - } - } - - private static class TFetchResultsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetHasMoreRows()) { - optionals.set(0); - } - if (struct.isSetResults()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetHasMoreRows()) { - oprot.writeBool(struct.hasMoreRows); - } - if (struct.isSetResults()) { - struct.results.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.hasMoreRows = iprot.readBool(); - struct.setHasMoreRowsIsSet(true); - } - if (incoming.get(1)) { - struct.results = new TRowSet(); - struct.results.read(iprot); - struct.setResultsIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java deleted file mode 100644 index cfd157f701b26..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetCatalogsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetCatalogsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetCatalogsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsReq.class, metaDataMap); - } - - public TGetCatalogsReq() { - } - - public TGetCatalogsReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetCatalogsReq(TGetCatalogsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetCatalogsReq deepCopy() { - return new TGetCatalogsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetCatalogsReq) - return this.equals((TGetCatalogsReq)that); - return false; - } - - public boolean equals(TGetCatalogsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetCatalogsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetCatalogsReq typedOther = (TGetCatalogsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetCatalogsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetCatalogsReqStandardSchemeFactory implements SchemeFactory { - public TGetCatalogsReqStandardScheme getScheme() { - return new TGetCatalogsReqStandardScheme(); - } - } - - private static class TGetCatalogsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetCatalogsReqTupleSchemeFactory implements SchemeFactory { - public TGetCatalogsReqTupleScheme getScheme() { - return new TGetCatalogsReqTupleScheme(); - } - } - - private static class TGetCatalogsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java deleted file mode 100644 index 1c5a35437d416..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetCatalogsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetCatalogsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetCatalogsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsResp.class, metaDataMap); - } - - public TGetCatalogsResp() { - } - - public TGetCatalogsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetCatalogsResp(TGetCatalogsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetCatalogsResp deepCopy() { - return new TGetCatalogsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetCatalogsResp) - return this.equals((TGetCatalogsResp)that); - return false; - } - - public boolean equals(TGetCatalogsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetCatalogsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetCatalogsResp typedOther = (TGetCatalogsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetCatalogsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetCatalogsRespStandardSchemeFactory implements SchemeFactory { - public TGetCatalogsRespStandardScheme getScheme() { - return new TGetCatalogsRespStandardScheme(); - } - } - - private static class TGetCatalogsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetCatalogsRespTupleSchemeFactory implements SchemeFactory { - public TGetCatalogsRespTupleScheme getScheme() { - return new TGetCatalogsRespTupleScheme(); - } - } - - private static class TGetCatalogsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java deleted file mode 100644 index a2c793bd95927..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java +++ /dev/null @@ -1,818 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetColumnsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); - private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetColumnsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetColumnsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String tableName; // optional - private String columnName; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - TABLE_NAME((short)4, "tableName"), - COLUMN_NAME((short)5, "columnName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // TABLE_NAME - return TABLE_NAME; - case 5: // COLUMN_NAME - return COLUMN_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.COLUMN_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsReq.class, metaDataMap); - } - - public TGetColumnsReq() { - } - - public TGetColumnsReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetColumnsReq(TGetColumnsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetTableName()) { - this.tableName = other.tableName; - } - if (other.isSetColumnName()) { - this.columnName = other.columnName; - } - } - - public TGetColumnsReq deepCopy() { - return new TGetColumnsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.tableName = null; - this.columnName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getTableName() { - return this.tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void unsetTableName() { - this.tableName = null; - } - - /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ - public boolean isSetTableName() { - return this.tableName != null; - } - - public void setTableNameIsSet(boolean value) { - if (!value) { - this.tableName = null; - } - } - - public String getColumnName() { - return this.columnName; - } - - public void setColumnName(String columnName) { - this.columnName = columnName; - } - - public void unsetColumnName() { - this.columnName = null; - } - - /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ - public boolean isSetColumnName() { - return this.columnName != null; - } - - public void setColumnNameIsSet(boolean value) { - if (!value) { - this.columnName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case TABLE_NAME: - if (value == null) { - unsetTableName(); - } else { - setTableName((String)value); - } - break; - - case COLUMN_NAME: - if (value == null) { - unsetColumnName(); - } else { - setColumnName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case TABLE_NAME: - return getTableName(); - - case COLUMN_NAME: - return getColumnName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case TABLE_NAME: - return isSetTableName(); - case COLUMN_NAME: - return isSetColumnName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetColumnsReq) - return this.equals((TGetColumnsReq)that); - return false; - } - - public boolean equals(TGetColumnsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_tableName = true && this.isSetTableName(); - boolean that_present_tableName = true && that.isSetTableName(); - if (this_present_tableName || that_present_tableName) { - if (!(this_present_tableName && that_present_tableName)) - return false; - if (!this.tableName.equals(that.tableName)) - return false; - } - - boolean this_present_columnName = true && this.isSetColumnName(); - boolean that_present_columnName = true && that.isSetColumnName(); - if (this_present_columnName || that_present_columnName) { - if (!(this_present_columnName && that_present_columnName)) - return false; - if (!this.columnName.equals(that.columnName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_tableName = true && (isSetTableName()); - builder.append(present_tableName); - if (present_tableName) - builder.append(tableName); - - boolean present_columnName = true && (isSetColumnName()); - builder.append(present_columnName); - if (present_columnName) - builder.append(columnName); - - return builder.toHashCode(); - } - - public int compareTo(TGetColumnsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetColumnsReq typedOther = (TGetColumnsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumnName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetColumnsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (isSetTableName()) { - if (!first) sb.append(", "); - sb.append("tableName:"); - if (this.tableName == null) { - sb.append("null"); - } else { - sb.append(this.tableName); - } - first = false; - } - if (isSetColumnName()) { - if (!first) sb.append(", "); - sb.append("columnName:"); - if (this.columnName == null) { - sb.append("null"); - } else { - sb.append(this.columnName); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetColumnsReqStandardSchemeFactory implements SchemeFactory { - public TGetColumnsReqStandardScheme getScheme() { - return new TGetColumnsReqStandardScheme(); - } - } - - private static class TGetColumnsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // TABLE_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // COLUMN_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.tableName != null) { - if (struct.isSetTableName()) { - oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); - oprot.writeString(struct.tableName); - oprot.writeFieldEnd(); - } - } - if (struct.columnName != null) { - if (struct.isSetColumnName()) { - oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); - oprot.writeString(struct.columnName); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetColumnsReqTupleSchemeFactory implements SchemeFactory { - public TGetColumnsReqTupleScheme getScheme() { - return new TGetColumnsReqTupleScheme(); - } - } - - private static class TGetColumnsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - if (struct.isSetTableName()) { - optionals.set(2); - } - if (struct.isSetColumnName()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - if (struct.isSetTableName()) { - oprot.writeString(struct.tableName); - } - if (struct.isSetColumnName()) { - oprot.writeString(struct.columnName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - if (incoming.get(2)) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } - if (incoming.get(3)) { - struct.columnName = iprot.readString(); - struct.setColumnNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java deleted file mode 100644 index d6cf1be6d304b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetColumnsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetColumnsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetColumnsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsResp.class, metaDataMap); - } - - public TGetColumnsResp() { - } - - public TGetColumnsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetColumnsResp(TGetColumnsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetColumnsResp deepCopy() { - return new TGetColumnsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetColumnsResp) - return this.equals((TGetColumnsResp)that); - return false; - } - - public boolean equals(TGetColumnsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetColumnsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetColumnsResp typedOther = (TGetColumnsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetColumnsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetColumnsRespStandardSchemeFactory implements SchemeFactory { - public TGetColumnsRespStandardScheme getScheme() { - return new TGetColumnsRespStandardScheme(); - } - } - - private static class TGetColumnsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetColumnsRespTupleSchemeFactory implements SchemeFactory { - public TGetColumnsRespTupleScheme getScheme() { - return new TGetColumnsRespTupleScheme(); - } - } - - private static class TGetColumnsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java deleted file mode 100644 index 6c6bb00e43e43..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java +++ /dev/null @@ -1,592 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField RENEWER_FIELD_DESC = new org.apache.thrift.protocol.TField("renewer", org.apache.thrift.protocol.TType.STRING, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String owner; // required - private String renewer; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - OWNER((short)2, "owner"), - RENEWER((short)3, "renewer"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // OWNER - return OWNER; - case 3: // RENEWER - return RENEWER; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.RENEWER, new org.apache.thrift.meta_data.FieldMetaData("renewer", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenReq.class, metaDataMap); - } - - public TGetDelegationTokenReq() { - } - - public TGetDelegationTokenReq( - TSessionHandle sessionHandle, - String owner, - String renewer) - { - this(); - this.sessionHandle = sessionHandle; - this.owner = owner; - this.renewer = renewer; - } - - /** - * Performs a deep copy on other. - */ - public TGetDelegationTokenReq(TGetDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetOwner()) { - this.owner = other.owner; - } - if (other.isSetRenewer()) { - this.renewer = other.renewer; - } - } - - public TGetDelegationTokenReq deepCopy() { - return new TGetDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.owner = null; - this.renewer = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getOwner() { - return this.owner; - } - - public void setOwner(String owner) { - this.owner = owner; - } - - public void unsetOwner() { - this.owner = null; - } - - /** Returns true if field owner is set (has been assigned a value) and false otherwise */ - public boolean isSetOwner() { - return this.owner != null; - } - - public void setOwnerIsSet(boolean value) { - if (!value) { - this.owner = null; - } - } - - public String getRenewer() { - return this.renewer; - } - - public void setRenewer(String renewer) { - this.renewer = renewer; - } - - public void unsetRenewer() { - this.renewer = null; - } - - /** Returns true if field renewer is set (has been assigned a value) and false otherwise */ - public boolean isSetRenewer() { - return this.renewer != null; - } - - public void setRenewerIsSet(boolean value) { - if (!value) { - this.renewer = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case OWNER: - if (value == null) { - unsetOwner(); - } else { - setOwner((String)value); - } - break; - - case RENEWER: - if (value == null) { - unsetRenewer(); - } else { - setRenewer((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case OWNER: - return getOwner(); - - case RENEWER: - return getRenewer(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case OWNER: - return isSetOwner(); - case RENEWER: - return isSetRenewer(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetDelegationTokenReq) - return this.equals((TGetDelegationTokenReq)that); - return false; - } - - public boolean equals(TGetDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_owner = true && this.isSetOwner(); - boolean that_present_owner = true && that.isSetOwner(); - if (this_present_owner || that_present_owner) { - if (!(this_present_owner && that_present_owner)) - return false; - if (!this.owner.equals(that.owner)) - return false; - } - - boolean this_present_renewer = true && this.isSetRenewer(); - boolean that_present_renewer = true && that.isSetRenewer(); - if (this_present_renewer || that_present_renewer) { - if (!(this_present_renewer && that_present_renewer)) - return false; - if (!this.renewer.equals(that.renewer)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_owner = true && (isSetOwner()); - builder.append(present_owner); - if (present_owner) - builder.append(owner); - - boolean present_renewer = true && (isSetRenewer()); - builder.append(present_renewer); - if (present_renewer) - builder.append(renewer); - - return builder.toHashCode(); - } - - public int compareTo(TGetDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetDelegationTokenReq typedOther = (TGetDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOwner()).compareTo(typedOther.isSetOwner()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOwner()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRenewer()).compareTo(typedOther.isSetRenewer()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRenewer()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.renewer, typedOther.renewer); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("owner:"); - if (this.owner == null) { - sb.append("null"); - } else { - sb.append(this.owner); - } - first = false; - if (!first) sb.append(", "); - sb.append("renewer:"); - if (this.renewer == null) { - sb.append("null"); - } else { - sb.append(this.renewer); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetOwner()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'owner' is unset! Struct:" + toString()); - } - - if (!isSetRenewer()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'renewer' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TGetDelegationTokenReqStandardScheme getScheme() { - return new TGetDelegationTokenReqStandardScheme(); - } - } - - private static class TGetDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OWNER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.owner = iprot.readString(); - struct.setOwnerIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // RENEWER - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.renewer = iprot.readString(); - struct.setRenewerIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.owner != null) { - oprot.writeFieldBegin(OWNER_FIELD_DESC); - oprot.writeString(struct.owner); - oprot.writeFieldEnd(); - } - if (struct.renewer != null) { - oprot.writeFieldBegin(RENEWER_FIELD_DESC); - oprot.writeString(struct.renewer); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TGetDelegationTokenReqTupleScheme getScheme() { - return new TGetDelegationTokenReqTupleScheme(); - } - } - - private static class TGetDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.owner); - oprot.writeString(struct.renewer); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.owner = iprot.readString(); - struct.setOwnerIsSet(true); - struct.renewer = iprot.readString(); - struct.setRenewerIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java deleted file mode 100644 index d14c5e029a35d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java +++ /dev/null @@ -1,500 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - private String delegationToken; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.DELEGATION_TOKEN}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenResp.class, metaDataMap); - } - - public TGetDelegationTokenResp() { - } - - public TGetDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetDelegationTokenResp(TGetDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TGetDelegationTokenResp deepCopy() { - return new TGetDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - this.delegationToken = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetDelegationTokenResp) - return this.equals((TGetDelegationTokenResp)that); - return false; - } - - public boolean equals(TGetDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TGetDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetDelegationTokenResp typedOther = (TGetDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetDelegationToken()) { - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TGetDelegationTokenRespStandardScheme getScheme() { - return new TGetDelegationTokenRespStandardScheme(); - } - } - - private static class TGetDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - if (struct.isSetDelegationToken()) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TGetDelegationTokenRespTupleScheme getScheme() { - return new TGetDelegationTokenRespTupleScheme(); - } - } - - private static class TGetDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetDelegationToken()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetDelegationToken()) { - oprot.writeString(struct.delegationToken); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java deleted file mode 100644 index ff45ee0386cb9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java +++ /dev/null @@ -1,707 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetFunctionsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetFunctionsReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetFunctionsReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String functionName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - FUNCTION_NAME((short)4, "functionName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // FUNCTION_NAME - return FUNCTION_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsReq.class, metaDataMap); - } - - public TGetFunctionsReq() { - } - - public TGetFunctionsReq( - TSessionHandle sessionHandle, - String functionName) - { - this(); - this.sessionHandle = sessionHandle; - this.functionName = functionName; - } - - /** - * Performs a deep copy on other. - */ - public TGetFunctionsReq(TGetFunctionsReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetFunctionName()) { - this.functionName = other.functionName; - } - } - - public TGetFunctionsReq deepCopy() { - return new TGetFunctionsReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.functionName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getFunctionName() { - return this.functionName; - } - - public void setFunctionName(String functionName) { - this.functionName = functionName; - } - - public void unsetFunctionName() { - this.functionName = null; - } - - /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ - public boolean isSetFunctionName() { - return this.functionName != null; - } - - public void setFunctionNameIsSet(boolean value) { - if (!value) { - this.functionName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case FUNCTION_NAME: - if (value == null) { - unsetFunctionName(); - } else { - setFunctionName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case FUNCTION_NAME: - return getFunctionName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case FUNCTION_NAME: - return isSetFunctionName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetFunctionsReq) - return this.equals((TGetFunctionsReq)that); - return false; - } - - public boolean equals(TGetFunctionsReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_functionName = true && this.isSetFunctionName(); - boolean that_present_functionName = true && that.isSetFunctionName(); - if (this_present_functionName || that_present_functionName) { - if (!(this_present_functionName && that_present_functionName)) - return false; - if (!this.functionName.equals(that.functionName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_functionName = true && (isSetFunctionName()); - builder.append(present_functionName); - if (present_functionName) - builder.append(functionName); - - return builder.toHashCode(); - } - - public int compareTo(TGetFunctionsReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetFunctionsReq typedOther = (TGetFunctionsReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetFunctionName()).compareTo(typedOther.isSetFunctionName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetFunctionName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, typedOther.functionName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetFunctionsReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (!first) sb.append(", "); - sb.append("functionName:"); - if (this.functionName == null) { - sb.append("null"); - } else { - sb.append(this.functionName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetFunctionName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'functionName' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetFunctionsReqStandardSchemeFactory implements SchemeFactory { - public TGetFunctionsReqStandardScheme getScheme() { - return new TGetFunctionsReqStandardScheme(); - } - } - - private static class TGetFunctionsReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // FUNCTION_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.functionName = iprot.readString(); - struct.setFunctionNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.functionName != null) { - oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); - oprot.writeString(struct.functionName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetFunctionsReqTupleSchemeFactory implements SchemeFactory { - public TGetFunctionsReqTupleScheme getScheme() { - return new TGetFunctionsReqTupleScheme(); - } - } - - private static class TGetFunctionsReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.functionName); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.functionName = iprot.readString(); - struct.setFunctionNameIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java deleted file mode 100644 index 3adafdacb54ef..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetFunctionsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetFunctionsRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetFunctionsRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsResp.class, metaDataMap); - } - - public TGetFunctionsResp() { - } - - public TGetFunctionsResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetFunctionsResp(TGetFunctionsResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetFunctionsResp deepCopy() { - return new TGetFunctionsResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetFunctionsResp) - return this.equals((TGetFunctionsResp)that); - return false; - } - - public boolean equals(TGetFunctionsResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetFunctionsResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetFunctionsResp typedOther = (TGetFunctionsResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetFunctionsResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetFunctionsRespStandardSchemeFactory implements SchemeFactory { - public TGetFunctionsRespStandardScheme getScheme() { - return new TGetFunctionsRespStandardScheme(); - } - } - - private static class TGetFunctionsRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetFunctionsRespTupleSchemeFactory implements SchemeFactory { - public TGetFunctionsRespTupleScheme getScheme() { - return new TGetFunctionsRespTupleScheme(); - } - } - - private static class TGetFunctionsRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java deleted file mode 100644 index 0139bf04ec7db..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java +++ /dev/null @@ -1,503 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField INFO_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoType", org.apache.thrift.protocol.TType.I32, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetInfoReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetInfoReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private TGetInfoType infoType; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - /** - * - * @see TGetInfoType - */ - INFO_TYPE((short)2, "infoType"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // INFO_TYPE - return INFO_TYPE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.INFO_TYPE, new org.apache.thrift.meta_data.FieldMetaData("infoType", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TGetInfoType.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoReq.class, metaDataMap); - } - - public TGetInfoReq() { - } - - public TGetInfoReq( - TSessionHandle sessionHandle, - TGetInfoType infoType) - { - this(); - this.sessionHandle = sessionHandle; - this.infoType = infoType; - } - - /** - * Performs a deep copy on other. - */ - public TGetInfoReq(TGetInfoReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetInfoType()) { - this.infoType = other.infoType; - } - } - - public TGetInfoReq deepCopy() { - return new TGetInfoReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.infoType = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - /** - * - * @see TGetInfoType - */ - public TGetInfoType getInfoType() { - return this.infoType; - } - - /** - * - * @see TGetInfoType - */ - public void setInfoType(TGetInfoType infoType) { - this.infoType = infoType; - } - - public void unsetInfoType() { - this.infoType = null; - } - - /** Returns true if field infoType is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoType() { - return this.infoType != null; - } - - public void setInfoTypeIsSet(boolean value) { - if (!value) { - this.infoType = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case INFO_TYPE: - if (value == null) { - unsetInfoType(); - } else { - setInfoType((TGetInfoType)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case INFO_TYPE: - return getInfoType(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case INFO_TYPE: - return isSetInfoType(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetInfoReq) - return this.equals((TGetInfoReq)that); - return false; - } - - public boolean equals(TGetInfoReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_infoType = true && this.isSetInfoType(); - boolean that_present_infoType = true && that.isSetInfoType(); - if (this_present_infoType || that_present_infoType) { - if (!(this_present_infoType && that_present_infoType)) - return false; - if (!this.infoType.equals(that.infoType)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_infoType = true && (isSetInfoType()); - builder.append(present_infoType); - if (present_infoType) - builder.append(infoType.getValue()); - - return builder.toHashCode(); - } - - public int compareTo(TGetInfoReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetInfoReq typedOther = (TGetInfoReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoType()).compareTo(typedOther.isSetInfoType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoType, typedOther.infoType); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetInfoReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("infoType:"); - if (this.infoType == null) { - sb.append("null"); - } else { - sb.append(this.infoType); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetInfoType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoType' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetInfoReqStandardSchemeFactory implements SchemeFactory { - public TGetInfoReqStandardScheme getScheme() { - return new TGetInfoReqStandardScheme(); - } - } - - private static class TGetInfoReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.infoType = TGetInfoType.findByValue(iprot.readI32()); - struct.setInfoTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.infoType != null) { - oprot.writeFieldBegin(INFO_TYPE_FIELD_DESC); - oprot.writeI32(struct.infoType.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetInfoReqTupleSchemeFactory implements SchemeFactory { - public TGetInfoReqTupleScheme getScheme() { - return new TGetInfoReqTupleScheme(); - } - } - - private static class TGetInfoReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeI32(struct.infoType.getValue()); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.infoType = TGetInfoType.findByValue(iprot.readI32()); - struct.setInfoTypeIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java deleted file mode 100644 index 2faaa9211b3ba..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java +++ /dev/null @@ -1,493 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField INFO_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoValue", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetInfoRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetInfoRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TGetInfoValue infoValue; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - INFO_VALUE((short)2, "infoValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // INFO_VALUE - return INFO_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.INFO_VALUE, new org.apache.thrift.meta_data.FieldMetaData("infoValue", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoValue.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoResp.class, metaDataMap); - } - - public TGetInfoResp() { - } - - public TGetInfoResp( - TStatus status, - TGetInfoValue infoValue) - { - this(); - this.status = status; - this.infoValue = infoValue; - } - - /** - * Performs a deep copy on other. - */ - public TGetInfoResp(TGetInfoResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetInfoValue()) { - this.infoValue = new TGetInfoValue(other.infoValue); - } - } - - public TGetInfoResp deepCopy() { - return new TGetInfoResp(this); - } - - @Override - public void clear() { - this.status = null; - this.infoValue = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TGetInfoValue getInfoValue() { - return this.infoValue; - } - - public void setInfoValue(TGetInfoValue infoValue) { - this.infoValue = infoValue; - } - - public void unsetInfoValue() { - this.infoValue = null; - } - - /** Returns true if field infoValue is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoValue() { - return this.infoValue != null; - } - - public void setInfoValueIsSet(boolean value) { - if (!value) { - this.infoValue = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case INFO_VALUE: - if (value == null) { - unsetInfoValue(); - } else { - setInfoValue((TGetInfoValue)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case INFO_VALUE: - return getInfoValue(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case INFO_VALUE: - return isSetInfoValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetInfoResp) - return this.equals((TGetInfoResp)that); - return false; - } - - public boolean equals(TGetInfoResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_infoValue = true && this.isSetInfoValue(); - boolean that_present_infoValue = true && that.isSetInfoValue(); - if (this_present_infoValue || that_present_infoValue) { - if (!(this_present_infoValue && that_present_infoValue)) - return false; - if (!this.infoValue.equals(that.infoValue)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_infoValue = true && (isSetInfoValue()); - builder.append(present_infoValue); - if (present_infoValue) - builder.append(infoValue); - - return builder.toHashCode(); - } - - public int compareTo(TGetInfoResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetInfoResp typedOther = (TGetInfoResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoValue()).compareTo(typedOther.isSetInfoValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoValue, typedOther.infoValue); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetInfoResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("infoValue:"); - if (this.infoValue == null) { - sb.append("null"); - } else { - sb.append(this.infoValue); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!isSetInfoValue()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoValue' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetInfoRespStandardSchemeFactory implements SchemeFactory { - public TGetInfoRespStandardScheme getScheme() { - return new TGetInfoRespStandardScheme(); - } - } - - private static class TGetInfoRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.infoValue = new TGetInfoValue(); - struct.infoValue.read(iprot); - struct.setInfoValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.infoValue != null) { - oprot.writeFieldBegin(INFO_VALUE_FIELD_DESC); - struct.infoValue.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetInfoRespTupleSchemeFactory implements SchemeFactory { - public TGetInfoRespTupleScheme getScheme() { - return new TGetInfoRespTupleScheme(); - } - } - - private static class TGetInfoRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - struct.infoValue.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - struct.infoValue = new TGetInfoValue(); - struct.infoValue.read(iprot); - struct.setInfoValueIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java deleted file mode 100644 index d9dd62414f001..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java +++ /dev/null @@ -1,180 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TGetInfoType implements org.apache.thrift.TEnum { - CLI_MAX_DRIVER_CONNECTIONS(0), - CLI_MAX_CONCURRENT_ACTIVITIES(1), - CLI_DATA_SOURCE_NAME(2), - CLI_FETCH_DIRECTION(8), - CLI_SERVER_NAME(13), - CLI_SEARCH_PATTERN_ESCAPE(14), - CLI_DBMS_NAME(17), - CLI_DBMS_VER(18), - CLI_ACCESSIBLE_TABLES(19), - CLI_ACCESSIBLE_PROCEDURES(20), - CLI_CURSOR_COMMIT_BEHAVIOR(23), - CLI_DATA_SOURCE_READ_ONLY(25), - CLI_DEFAULT_TXN_ISOLATION(26), - CLI_IDENTIFIER_CASE(28), - CLI_IDENTIFIER_QUOTE_CHAR(29), - CLI_MAX_COLUMN_NAME_LEN(30), - CLI_MAX_CURSOR_NAME_LEN(31), - CLI_MAX_SCHEMA_NAME_LEN(32), - CLI_MAX_CATALOG_NAME_LEN(34), - CLI_MAX_TABLE_NAME_LEN(35), - CLI_SCROLL_CONCURRENCY(43), - CLI_TXN_CAPABLE(46), - CLI_USER_NAME(47), - CLI_TXN_ISOLATION_OPTION(72), - CLI_INTEGRITY(73), - CLI_GETDATA_EXTENSIONS(81), - CLI_NULL_COLLATION(85), - CLI_ALTER_TABLE(86), - CLI_ORDER_BY_COLUMNS_IN_SELECT(90), - CLI_SPECIAL_CHARACTERS(94), - CLI_MAX_COLUMNS_IN_GROUP_BY(97), - CLI_MAX_COLUMNS_IN_INDEX(98), - CLI_MAX_COLUMNS_IN_ORDER_BY(99), - CLI_MAX_COLUMNS_IN_SELECT(100), - CLI_MAX_COLUMNS_IN_TABLE(101), - CLI_MAX_INDEX_SIZE(102), - CLI_MAX_ROW_SIZE(104), - CLI_MAX_STATEMENT_LEN(105), - CLI_MAX_TABLES_IN_SELECT(106), - CLI_MAX_USER_NAME_LEN(107), - CLI_OJ_CAPABILITIES(115), - CLI_XOPEN_CLI_YEAR(10000), - CLI_CURSOR_SENSITIVITY(10001), - CLI_DESCRIBE_PARAMETER(10002), - CLI_CATALOG_NAME(10003), - CLI_COLLATION_SEQ(10004), - CLI_MAX_IDENTIFIER_LEN(10005); - - private final int value; - - private TGetInfoType(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TGetInfoType findByValue(int value) { - switch (value) { - case 0: - return CLI_MAX_DRIVER_CONNECTIONS; - case 1: - return CLI_MAX_CONCURRENT_ACTIVITIES; - case 2: - return CLI_DATA_SOURCE_NAME; - case 8: - return CLI_FETCH_DIRECTION; - case 13: - return CLI_SERVER_NAME; - case 14: - return CLI_SEARCH_PATTERN_ESCAPE; - case 17: - return CLI_DBMS_NAME; - case 18: - return CLI_DBMS_VER; - case 19: - return CLI_ACCESSIBLE_TABLES; - case 20: - return CLI_ACCESSIBLE_PROCEDURES; - case 23: - return CLI_CURSOR_COMMIT_BEHAVIOR; - case 25: - return CLI_DATA_SOURCE_READ_ONLY; - case 26: - return CLI_DEFAULT_TXN_ISOLATION; - case 28: - return CLI_IDENTIFIER_CASE; - case 29: - return CLI_IDENTIFIER_QUOTE_CHAR; - case 30: - return CLI_MAX_COLUMN_NAME_LEN; - case 31: - return CLI_MAX_CURSOR_NAME_LEN; - case 32: - return CLI_MAX_SCHEMA_NAME_LEN; - case 34: - return CLI_MAX_CATALOG_NAME_LEN; - case 35: - return CLI_MAX_TABLE_NAME_LEN; - case 43: - return CLI_SCROLL_CONCURRENCY; - case 46: - return CLI_TXN_CAPABLE; - case 47: - return CLI_USER_NAME; - case 72: - return CLI_TXN_ISOLATION_OPTION; - case 73: - return CLI_INTEGRITY; - case 81: - return CLI_GETDATA_EXTENSIONS; - case 85: - return CLI_NULL_COLLATION; - case 86: - return CLI_ALTER_TABLE; - case 90: - return CLI_ORDER_BY_COLUMNS_IN_SELECT; - case 94: - return CLI_SPECIAL_CHARACTERS; - case 97: - return CLI_MAX_COLUMNS_IN_GROUP_BY; - case 98: - return CLI_MAX_COLUMNS_IN_INDEX; - case 99: - return CLI_MAX_COLUMNS_IN_ORDER_BY; - case 100: - return CLI_MAX_COLUMNS_IN_SELECT; - case 101: - return CLI_MAX_COLUMNS_IN_TABLE; - case 102: - return CLI_MAX_INDEX_SIZE; - case 104: - return CLI_MAX_ROW_SIZE; - case 105: - return CLI_MAX_STATEMENT_LEN; - case 106: - return CLI_MAX_TABLES_IN_SELECT; - case 107: - return CLI_MAX_USER_NAME_LEN; - case 115: - return CLI_OJ_CAPABILITIES; - case 10000: - return CLI_XOPEN_CLI_YEAR; - case 10001: - return CLI_CURSOR_SENSITIVITY; - case 10002: - return CLI_DESCRIBE_PARAMETER; - case 10003: - return CLI_CATALOG_NAME; - case 10004: - return CLI_COLLATION_SEQ; - case 10005: - return CLI_MAX_IDENTIFIER_LEN; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java deleted file mode 100644 index fe2a211c46309..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java +++ /dev/null @@ -1,593 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetInfoValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoValue"); - private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField SMALL_INT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("smallIntValue", org.apache.thrift.protocol.TType.I16, (short)2); - private static final org.apache.thrift.protocol.TField INTEGER_BITMASK_FIELD_DESC = new org.apache.thrift.protocol.TField("integerBitmask", org.apache.thrift.protocol.TType.I32, (short)3); - private static final org.apache.thrift.protocol.TField INTEGER_FLAG_FIELD_DESC = new org.apache.thrift.protocol.TField("integerFlag", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField BINARY_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryValue", org.apache.thrift.protocol.TType.I32, (short)5); - private static final org.apache.thrift.protocol.TField LEN_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lenValue", org.apache.thrift.protocol.TType.I64, (short)6); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STRING_VALUE((short)1, "stringValue"), - SMALL_INT_VALUE((short)2, "smallIntValue"), - INTEGER_BITMASK((short)3, "integerBitmask"), - INTEGER_FLAG((short)4, "integerFlag"), - BINARY_VALUE((short)5, "binaryValue"), - LEN_VALUE((short)6, "lenValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STRING_VALUE - return STRING_VALUE; - case 2: // SMALL_INT_VALUE - return SMALL_INT_VALUE; - case 3: // INTEGER_BITMASK - return INTEGER_BITMASK; - case 4: // INTEGER_FLAG - return INTEGER_FLAG; - case 5: // BINARY_VALUE - return BINARY_VALUE; - case 6: // LEN_VALUE - return LEN_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.SMALL_INT_VALUE, new org.apache.thrift.meta_data.FieldMetaData("smallIntValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - tmpMap.put(_Fields.INTEGER_BITMASK, new org.apache.thrift.meta_data.FieldMetaData("integerBitmask", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.INTEGER_FLAG, new org.apache.thrift.meta_data.FieldMetaData("integerFlag", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.BINARY_VALUE, new org.apache.thrift.meta_data.FieldMetaData("binaryValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.LEN_VALUE, new org.apache.thrift.meta_data.FieldMetaData("lenValue", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoValue.class, metaDataMap); - } - - public TGetInfoValue() { - super(); - } - - public TGetInfoValue(TGetInfoValue._Fields setField, Object value) { - super(setField, value); - } - - public TGetInfoValue(TGetInfoValue other) { - super(other); - } - public TGetInfoValue deepCopy() { - return new TGetInfoValue(this); - } - - public static TGetInfoValue stringValue(String value) { - TGetInfoValue x = new TGetInfoValue(); - x.setStringValue(value); - return x; - } - - public static TGetInfoValue smallIntValue(short value) { - TGetInfoValue x = new TGetInfoValue(); - x.setSmallIntValue(value); - return x; - } - - public static TGetInfoValue integerBitmask(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setIntegerBitmask(value); - return x; - } - - public static TGetInfoValue integerFlag(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setIntegerFlag(value); - return x; - } - - public static TGetInfoValue binaryValue(int value) { - TGetInfoValue x = new TGetInfoValue(); - x.setBinaryValue(value); - return x; - } - - public static TGetInfoValue lenValue(long value) { - TGetInfoValue x = new TGetInfoValue(); - x.setLenValue(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case STRING_VALUE: - if (value instanceof String) { - break; - } - throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); - case SMALL_INT_VALUE: - if (value instanceof Short) { - break; - } - throw new ClassCastException("Was expecting value of type Short for field 'smallIntValue', but got " + value.getClass().getSimpleName()); - case INTEGER_BITMASK: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'integerBitmask', but got " + value.getClass().getSimpleName()); - case INTEGER_FLAG: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'integerFlag', but got " + value.getClass().getSimpleName()); - case BINARY_VALUE: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'binaryValue', but got " + value.getClass().getSimpleName()); - case LEN_VALUE: - if (value instanceof Long) { - break; - } - throw new ClassCastException("Was expecting value of type Long for field 'lenValue', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case STRING_VALUE: - if (field.type == STRING_VALUE_FIELD_DESC.type) { - String stringValue; - stringValue = iprot.readString(); - return stringValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case SMALL_INT_VALUE: - if (field.type == SMALL_INT_VALUE_FIELD_DESC.type) { - Short smallIntValue; - smallIntValue = iprot.readI16(); - return smallIntValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case INTEGER_BITMASK: - if (field.type == INTEGER_BITMASK_FIELD_DESC.type) { - Integer integerBitmask; - integerBitmask = iprot.readI32(); - return integerBitmask; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case INTEGER_FLAG: - if (field.type == INTEGER_FLAG_FIELD_DESC.type) { - Integer integerFlag; - integerFlag = iprot.readI32(); - return integerFlag; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BINARY_VALUE: - if (field.type == BINARY_VALUE_FIELD_DESC.type) { - Integer binaryValue; - binaryValue = iprot.readI32(); - return binaryValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case LEN_VALUE: - if (field.type == LEN_VALUE_FIELD_DESC.type) { - Long lenValue; - lenValue = iprot.readI64(); - return lenValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - case SMALL_INT_VALUE: - Short smallIntValue = (Short)value_; - oprot.writeI16(smallIntValue); - return; - case INTEGER_BITMASK: - Integer integerBitmask = (Integer)value_; - oprot.writeI32(integerBitmask); - return; - case INTEGER_FLAG: - Integer integerFlag = (Integer)value_; - oprot.writeI32(integerFlag); - return; - case BINARY_VALUE: - Integer binaryValue = (Integer)value_; - oprot.writeI32(binaryValue); - return; - case LEN_VALUE: - Long lenValue = (Long)value_; - oprot.writeI64(lenValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case STRING_VALUE: - String stringValue; - stringValue = iprot.readString(); - return stringValue; - case SMALL_INT_VALUE: - Short smallIntValue; - smallIntValue = iprot.readI16(); - return smallIntValue; - case INTEGER_BITMASK: - Integer integerBitmask; - integerBitmask = iprot.readI32(); - return integerBitmask; - case INTEGER_FLAG: - Integer integerFlag; - integerFlag = iprot.readI32(); - return integerFlag; - case BINARY_VALUE: - Integer binaryValue; - binaryValue = iprot.readI32(); - return binaryValue; - case LEN_VALUE: - Long lenValue; - lenValue = iprot.readI64(); - return lenValue; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - case SMALL_INT_VALUE: - Short smallIntValue = (Short)value_; - oprot.writeI16(smallIntValue); - return; - case INTEGER_BITMASK: - Integer integerBitmask = (Integer)value_; - oprot.writeI32(integerBitmask); - return; - case INTEGER_FLAG: - Integer integerFlag = (Integer)value_; - oprot.writeI32(integerFlag); - return; - case BINARY_VALUE: - Integer binaryValue = (Integer)value_; - oprot.writeI32(binaryValue); - return; - case LEN_VALUE: - Long lenValue = (Long)value_; - oprot.writeI64(lenValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case STRING_VALUE: - return STRING_VALUE_FIELD_DESC; - case SMALL_INT_VALUE: - return SMALL_INT_VALUE_FIELD_DESC; - case INTEGER_BITMASK: - return INTEGER_BITMASK_FIELD_DESC; - case INTEGER_FLAG: - return INTEGER_FLAG_FIELD_DESC; - case BINARY_VALUE: - return BINARY_VALUE_FIELD_DESC; - case LEN_VALUE: - return LEN_VALUE_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public String getStringValue() { - if (getSetField() == _Fields.STRING_VALUE) { - return (String)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringValue(String value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VALUE; - value_ = value; - } - - public short getSmallIntValue() { - if (getSetField() == _Fields.SMALL_INT_VALUE) { - return (Short)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'smallIntValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setSmallIntValue(short value) { - setField_ = _Fields.SMALL_INT_VALUE; - value_ = value; - } - - public int getIntegerBitmask() { - if (getSetField() == _Fields.INTEGER_BITMASK) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'integerBitmask' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setIntegerBitmask(int value) { - setField_ = _Fields.INTEGER_BITMASK; - value_ = value; - } - - public int getIntegerFlag() { - if (getSetField() == _Fields.INTEGER_FLAG) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'integerFlag' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setIntegerFlag(int value) { - setField_ = _Fields.INTEGER_FLAG; - value_ = value; - } - - public int getBinaryValue() { - if (getSetField() == _Fields.BINARY_VALUE) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'binaryValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setBinaryValue(int value) { - setField_ = _Fields.BINARY_VALUE; - value_ = value; - } - - public long getLenValue() { - if (getSetField() == _Fields.LEN_VALUE) { - return (Long)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'lenValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setLenValue(long value) { - setField_ = _Fields.LEN_VALUE; - value_ = value; - } - - public boolean isSetStringValue() { - return setField_ == _Fields.STRING_VALUE; - } - - - public boolean isSetSmallIntValue() { - return setField_ == _Fields.SMALL_INT_VALUE; - } - - - public boolean isSetIntegerBitmask() { - return setField_ == _Fields.INTEGER_BITMASK; - } - - - public boolean isSetIntegerFlag() { - return setField_ == _Fields.INTEGER_FLAG; - } - - - public boolean isSetBinaryValue() { - return setField_ == _Fields.BINARY_VALUE; - } - - - public boolean isSetLenValue() { - return setField_ == _Fields.LEN_VALUE; - } - - - public boolean equals(Object other) { - if (other instanceof TGetInfoValue) { - return equals((TGetInfoValue)other); - } else { - return false; - } - } - - public boolean equals(TGetInfoValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TGetInfoValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java deleted file mode 100644 index b88591ea1945b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetOperationStatusReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetOperationStatusReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetOperationStatusReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusReq.class, metaDataMap); - } - - public TGetOperationStatusReq() { - } - - public TGetOperationStatusReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetOperationStatusReq(TGetOperationStatusReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetOperationStatusReq deepCopy() { - return new TGetOperationStatusReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetOperationStatusReq) - return this.equals((TGetOperationStatusReq)that); - return false; - } - - public boolean equals(TGetOperationStatusReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetOperationStatusReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetOperationStatusReq typedOther = (TGetOperationStatusReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetOperationStatusReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetOperationStatusReqStandardSchemeFactory implements SchemeFactory { - public TGetOperationStatusReqStandardScheme getScheme() { - return new TGetOperationStatusReqStandardScheme(); - } - } - - private static class TGetOperationStatusReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetOperationStatusReqTupleSchemeFactory implements SchemeFactory { - public TGetOperationStatusReqTupleScheme getScheme() { - return new TGetOperationStatusReqTupleScheme(); - } - } - - private static class TGetOperationStatusReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java deleted file mode 100644 index 94ba6bb1146de..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java +++ /dev/null @@ -1,827 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetOperationStatusResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationState", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetOperationStatusRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetOperationStatusRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationState operationState; // optional - private String sqlState; // optional - private int errorCode; // optional - private String errorMessage; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - /** - * - * @see TOperationState - */ - OPERATION_STATE((short)2, "operationState"), - SQL_STATE((short)3, "sqlState"), - ERROR_CODE((short)4, "errorCode"), - ERROR_MESSAGE((short)5, "errorMessage"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_STATE - return OPERATION_STATE; - case 3: // SQL_STATE - return SQL_STATE; - case 4: // ERROR_CODE - return ERROR_CODE; - case 5: // ERROR_MESSAGE - return ERROR_MESSAGE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ERRORCODE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.OPERATION_STATE,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_STATE, new org.apache.thrift.meta_data.FieldMetaData("operationState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationState.class))); - tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusResp.class, metaDataMap); - } - - public TGetOperationStatusResp() { - } - - public TGetOperationStatusResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetOperationStatusResp(TGetOperationStatusResp other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationState()) { - this.operationState = other.operationState; - } - if (other.isSetSqlState()) { - this.sqlState = other.sqlState; - } - this.errorCode = other.errorCode; - if (other.isSetErrorMessage()) { - this.errorMessage = other.errorMessage; - } - } - - public TGetOperationStatusResp deepCopy() { - return new TGetOperationStatusResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationState = null; - this.sqlState = null; - setErrorCodeIsSet(false); - this.errorCode = 0; - this.errorMessage = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - /** - * - * @see TOperationState - */ - public TOperationState getOperationState() { - return this.operationState; - } - - /** - * - * @see TOperationState - */ - public void setOperationState(TOperationState operationState) { - this.operationState = operationState; - } - - public void unsetOperationState() { - this.operationState = null; - } - - /** Returns true if field operationState is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationState() { - return this.operationState != null; - } - - public void setOperationStateIsSet(boolean value) { - if (!value) { - this.operationState = null; - } - } - - public String getSqlState() { - return this.sqlState; - } - - public void setSqlState(String sqlState) { - this.sqlState = sqlState; - } - - public void unsetSqlState() { - this.sqlState = null; - } - - /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ - public boolean isSetSqlState() { - return this.sqlState != null; - } - - public void setSqlStateIsSet(boolean value) { - if (!value) { - this.sqlState = null; - } - } - - public int getErrorCode() { - return this.errorCode; - } - - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - setErrorCodeIsSet(true); - } - - public void unsetErrorCode() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorCode() { - return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - public void setErrorCodeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); - } - - public String getErrorMessage() { - return this.errorMessage; - } - - public void setErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - } - - public void unsetErrorMessage() { - this.errorMessage = null; - } - - /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorMessage() { - return this.errorMessage != null; - } - - public void setErrorMessageIsSet(boolean value) { - if (!value) { - this.errorMessage = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_STATE: - if (value == null) { - unsetOperationState(); - } else { - setOperationState((TOperationState)value); - } - break; - - case SQL_STATE: - if (value == null) { - unsetSqlState(); - } else { - setSqlState((String)value); - } - break; - - case ERROR_CODE: - if (value == null) { - unsetErrorCode(); - } else { - setErrorCode((Integer)value); - } - break; - - case ERROR_MESSAGE: - if (value == null) { - unsetErrorMessage(); - } else { - setErrorMessage((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_STATE: - return getOperationState(); - - case SQL_STATE: - return getSqlState(); - - case ERROR_CODE: - return Integer.valueOf(getErrorCode()); - - case ERROR_MESSAGE: - return getErrorMessage(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_STATE: - return isSetOperationState(); - case SQL_STATE: - return isSetSqlState(); - case ERROR_CODE: - return isSetErrorCode(); - case ERROR_MESSAGE: - return isSetErrorMessage(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetOperationStatusResp) - return this.equals((TGetOperationStatusResp)that); - return false; - } - - public boolean equals(TGetOperationStatusResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationState = true && this.isSetOperationState(); - boolean that_present_operationState = true && that.isSetOperationState(); - if (this_present_operationState || that_present_operationState) { - if (!(this_present_operationState && that_present_operationState)) - return false; - if (!this.operationState.equals(that.operationState)) - return false; - } - - boolean this_present_sqlState = true && this.isSetSqlState(); - boolean that_present_sqlState = true && that.isSetSqlState(); - if (this_present_sqlState || that_present_sqlState) { - if (!(this_present_sqlState && that_present_sqlState)) - return false; - if (!this.sqlState.equals(that.sqlState)) - return false; - } - - boolean this_present_errorCode = true && this.isSetErrorCode(); - boolean that_present_errorCode = true && that.isSetErrorCode(); - if (this_present_errorCode || that_present_errorCode) { - if (!(this_present_errorCode && that_present_errorCode)) - return false; - if (this.errorCode != that.errorCode) - return false; - } - - boolean this_present_errorMessage = true && this.isSetErrorMessage(); - boolean that_present_errorMessage = true && that.isSetErrorMessage(); - if (this_present_errorMessage || that_present_errorMessage) { - if (!(this_present_errorMessage && that_present_errorMessage)) - return false; - if (!this.errorMessage.equals(that.errorMessage)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationState = true && (isSetOperationState()); - builder.append(present_operationState); - if (present_operationState) - builder.append(operationState.getValue()); - - boolean present_sqlState = true && (isSetSqlState()); - builder.append(present_sqlState); - if (present_sqlState) - builder.append(sqlState); - - boolean present_errorCode = true && (isSetErrorCode()); - builder.append(present_errorCode); - if (present_errorCode) - builder.append(errorCode); - - boolean present_errorMessage = true && (isSetErrorMessage()); - builder.append(present_errorMessage); - if (present_errorMessage) - builder.append(errorMessage); - - return builder.toHashCode(); - } - - public int compareTo(TGetOperationStatusResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetOperationStatusResp typedOther = (TGetOperationStatusResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationState()).compareTo(typedOther.isSetOperationState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationState, typedOther.operationState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSqlState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetOperationStatusResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationState()) { - if (!first) sb.append(", "); - sb.append("operationState:"); - if (this.operationState == null) { - sb.append("null"); - } else { - sb.append(this.operationState); - } - first = false; - } - if (isSetSqlState()) { - if (!first) sb.append(", "); - sb.append("sqlState:"); - if (this.sqlState == null) { - sb.append("null"); - } else { - sb.append(this.sqlState); - } - first = false; - } - if (isSetErrorCode()) { - if (!first) sb.append(", "); - sb.append("errorCode:"); - sb.append(this.errorCode); - first = false; - } - if (isSetErrorMessage()) { - if (!first) sb.append(", "); - sb.append("errorMessage:"); - if (this.errorMessage == null) { - sb.append("null"); - } else { - sb.append(this.errorMessage); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetOperationStatusRespStandardSchemeFactory implements SchemeFactory { - public TGetOperationStatusRespStandardScheme getScheme() { - return new TGetOperationStatusRespStandardScheme(); - } - } - - private static class TGetOperationStatusRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.operationState = TOperationState.findByValue(iprot.readI32()); - struct.setOperationStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SQL_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // ERROR_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // ERROR_MESSAGE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationState != null) { - if (struct.isSetOperationState()) { - oprot.writeFieldBegin(OPERATION_STATE_FIELD_DESC); - oprot.writeI32(struct.operationState.getValue()); - oprot.writeFieldEnd(); - } - } - if (struct.sqlState != null) { - if (struct.isSetSqlState()) { - oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); - oprot.writeString(struct.sqlState); - oprot.writeFieldEnd(); - } - } - if (struct.isSetErrorCode()) { - oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); - oprot.writeI32(struct.errorCode); - oprot.writeFieldEnd(); - } - if (struct.errorMessage != null) { - if (struct.isSetErrorMessage()) { - oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); - oprot.writeString(struct.errorMessage); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetOperationStatusRespTupleSchemeFactory implements SchemeFactory { - public TGetOperationStatusRespTupleScheme getScheme() { - return new TGetOperationStatusRespTupleScheme(); - } - } - - private static class TGetOperationStatusRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationState()) { - optionals.set(0); - } - if (struct.isSetSqlState()) { - optionals.set(1); - } - if (struct.isSetErrorCode()) { - optionals.set(2); - } - if (struct.isSetErrorMessage()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetOperationState()) { - oprot.writeI32(struct.operationState.getValue()); - } - if (struct.isSetSqlState()) { - oprot.writeString(struct.sqlState); - } - if (struct.isSetErrorCode()) { - oprot.writeI32(struct.errorCode); - } - if (struct.isSetErrorMessage()) { - oprot.writeString(struct.errorMessage); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.operationState = TOperationState.findByValue(iprot.readI32()); - struct.setOperationStateIsSet(true); - } - if (incoming.get(1)) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } - if (incoming.get(2)) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } - if (incoming.get(3)) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java deleted file mode 100644 index 3bf363c958468..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetResultSetMetadataReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataReq"); - - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetResultSetMetadataReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetResultSetMetadataReqTupleSchemeFactory()); - } - - private TOperationHandle operationHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_HANDLE((short)1, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataReq.class, metaDataMap); - } - - public TGetResultSetMetadataReq() { - } - - public TGetResultSetMetadataReq( - TOperationHandle operationHandle) - { - this(); - this.operationHandle = operationHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetResultSetMetadataReq(TGetResultSetMetadataReq other) { - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetResultSetMetadataReq deepCopy() { - return new TGetResultSetMetadataReq(this); - } - - @Override - public void clear() { - this.operationHandle = null; - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetResultSetMetadataReq) - return this.equals((TGetResultSetMetadataReq)that); - return false; - } - - public boolean equals(TGetResultSetMetadataReq that) { - if (that == null) - return false; - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetResultSetMetadataReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetResultSetMetadataReq typedOther = (TGetResultSetMetadataReq)other; - - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetResultSetMetadataReq("); - boolean first = true; - - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetResultSetMetadataReqStandardSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataReqStandardScheme getScheme() { - return new TGetResultSetMetadataReqStandardScheme(); - } - } - - private static class TGetResultSetMetadataReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationHandle != null) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetResultSetMetadataReqTupleSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataReqTupleScheme getScheme() { - return new TGetResultSetMetadataReqTupleScheme(); - } - } - - private static class TGetResultSetMetadataReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java deleted file mode 100644 index a9bef9f722c16..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetResultSetMetadataResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetResultSetMetadataRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetResultSetMetadataRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TTableSchema schema; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - SCHEMA((short)2, "schema"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // SCHEMA - return SCHEMA; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.SCHEMA}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableSchema.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataResp.class, metaDataMap); - } - - public TGetResultSetMetadataResp() { - } - - public TGetResultSetMetadataResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetResultSetMetadataResp(TGetResultSetMetadataResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetSchema()) { - this.schema = new TTableSchema(other.schema); - } - } - - public TGetResultSetMetadataResp deepCopy() { - return new TGetResultSetMetadataResp(this); - } - - @Override - public void clear() { - this.status = null; - this.schema = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TTableSchema getSchema() { - return this.schema; - } - - public void setSchema(TTableSchema schema) { - this.schema = schema; - } - - public void unsetSchema() { - this.schema = null; - } - - /** Returns true if field schema is set (has been assigned a value) and false otherwise */ - public boolean isSetSchema() { - return this.schema != null; - } - - public void setSchemaIsSet(boolean value) { - if (!value) { - this.schema = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case SCHEMA: - if (value == null) { - unsetSchema(); - } else { - setSchema((TTableSchema)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case SCHEMA: - return getSchema(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case SCHEMA: - return isSetSchema(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetResultSetMetadataResp) - return this.equals((TGetResultSetMetadataResp)that); - return false; - } - - public boolean equals(TGetResultSetMetadataResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_schema = true && this.isSetSchema(); - boolean that_present_schema = true && that.isSetSchema(); - if (this_present_schema || that_present_schema) { - if (!(this_present_schema && that_present_schema)) - return false; - if (!this.schema.equals(that.schema)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_schema = true && (isSetSchema()); - builder.append(present_schema); - if (present_schema) - builder.append(schema); - - return builder.toHashCode(); - } - - public int compareTo(TGetResultSetMetadataResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetResultSetMetadataResp typedOther = (TGetResultSetMetadataResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchema()).compareTo(typedOther.isSetSchema()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchema()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, typedOther.schema); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetResultSetMetadataResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetSchema()) { - if (!first) sb.append(", "); - sb.append("schema:"); - if (this.schema == null) { - sb.append("null"); - } else { - sb.append(this.schema); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (schema != null) { - schema.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetResultSetMetadataRespStandardSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataRespStandardScheme getScheme() { - return new TGetResultSetMetadataRespStandardScheme(); - } - } - - private static class TGetResultSetMetadataRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SCHEMA - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.schema = new TTableSchema(); - struct.schema.read(iprot); - struct.setSchemaIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.schema != null) { - if (struct.isSetSchema()) { - oprot.writeFieldBegin(SCHEMA_FIELD_DESC); - struct.schema.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetResultSetMetadataRespTupleSchemeFactory implements SchemeFactory { - public TGetResultSetMetadataRespTupleScheme getScheme() { - return new TGetResultSetMetadataRespTupleScheme(); - } - } - - private static class TGetResultSetMetadataRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetSchema()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSchema()) { - struct.schema.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.schema = new TTableSchema(); - struct.schema.read(iprot); - struct.setSchemaIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java deleted file mode 100644 index c2aadaa49a1e9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java +++ /dev/null @@ -1,606 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetSchemasReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetSchemasReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetSchemasReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasReq.class, metaDataMap); - } - - public TGetSchemasReq() { - } - - public TGetSchemasReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetSchemasReq(TGetSchemasReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - } - - public TGetSchemasReq deepCopy() { - return new TGetSchemasReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetSchemasReq) - return this.equals((TGetSchemasReq)that); - return false; - } - - public boolean equals(TGetSchemasReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - return builder.toHashCode(); - } - - public int compareTo(TGetSchemasReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetSchemasReq typedOther = (TGetSchemasReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetSchemasReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetSchemasReqStandardSchemeFactory implements SchemeFactory { - public TGetSchemasReqStandardScheme getScheme() { - return new TGetSchemasReqStandardScheme(); - } - } - - private static class TGetSchemasReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetSchemasReqTupleSchemeFactory implements SchemeFactory { - public TGetSchemasReqTupleScheme getScheme() { - return new TGetSchemasReqTupleScheme(); - } - } - - private static class TGetSchemasReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java deleted file mode 100644 index ac1ea3e7cc7af..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetSchemasResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetSchemasRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetSchemasRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasResp.class, metaDataMap); - } - - public TGetSchemasResp() { - } - - public TGetSchemasResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetSchemasResp(TGetSchemasResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetSchemasResp deepCopy() { - return new TGetSchemasResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetSchemasResp) - return this.equals((TGetSchemasResp)that); - return false; - } - - public boolean equals(TGetSchemasResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetSchemasResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetSchemasResp typedOther = (TGetSchemasResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetSchemasResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetSchemasRespStandardSchemeFactory implements SchemeFactory { - public TGetSchemasRespStandardScheme getScheme() { - return new TGetSchemasRespStandardScheme(); - } - } - - private static class TGetSchemasRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetSchemasRespTupleSchemeFactory implements SchemeFactory { - public TGetSchemasRespTupleScheme getScheme() { - return new TGetSchemasRespTupleScheme(); - } - } - - private static class TGetSchemasRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java deleted file mode 100644 index 6f2c713e0be6a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTableTypesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTableTypesReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTableTypesReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesReq.class, metaDataMap); - } - - public TGetTableTypesReq() { - } - - public TGetTableTypesReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTableTypesReq(TGetTableTypesReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetTableTypesReq deepCopy() { - return new TGetTableTypesReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTableTypesReq) - return this.equals((TGetTableTypesReq)that); - return false; - } - - public boolean equals(TGetTableTypesReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTableTypesReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTableTypesReq typedOther = (TGetTableTypesReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTableTypesReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTableTypesReqStandardSchemeFactory implements SchemeFactory { - public TGetTableTypesReqStandardScheme getScheme() { - return new TGetTableTypesReqStandardScheme(); - } - } - - private static class TGetTableTypesReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTableTypesReqTupleSchemeFactory implements SchemeFactory { - public TGetTableTypesReqTupleScheme getScheme() { - return new TGetTableTypesReqTupleScheme(); - } - } - - private static class TGetTableTypesReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java deleted file mode 100644 index 6f33fbcf5dadc..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTableTypesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTableTypesRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTableTypesRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesResp.class, metaDataMap); - } - - public TGetTableTypesResp() { - } - - public TGetTableTypesResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTableTypesResp(TGetTableTypesResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTableTypesResp deepCopy() { - return new TGetTableTypesResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTableTypesResp) - return this.equals((TGetTableTypesResp)that); - return false; - } - - public boolean equals(TGetTableTypesResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTableTypesResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTableTypesResp typedOther = (TGetTableTypesResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTableTypesResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTableTypesRespStandardSchemeFactory implements SchemeFactory { - public TGetTableTypesRespStandardScheme getScheme() { - return new TGetTableTypesRespStandardScheme(); - } - } - - private static class TGetTableTypesRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTableTypesRespTupleSchemeFactory implements SchemeFactory { - public TGetTableTypesRespTupleScheme getScheme() { - return new TGetTableTypesRespTupleScheme(); - } - } - - private static class TGetTableTypesRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java deleted file mode 100644 index c973fcc24cb10..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java +++ /dev/null @@ -1,870 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTablesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); - private static final org.apache.thrift.protocol.TField TABLE_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableTypes", org.apache.thrift.protocol.TType.LIST, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTablesReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTablesReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String catalogName; // optional - private String schemaName; // optional - private String tableName; // optional - private List tableTypes; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - CATALOG_NAME((short)2, "catalogName"), - SCHEMA_NAME((short)3, "schemaName"), - TABLE_NAME((short)4, "tableName"), - TABLE_TYPES((short)5, "tableTypes"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // CATALOG_NAME - return CATALOG_NAME; - case 3: // SCHEMA_NAME - return SCHEMA_NAME; - case 4: // TABLE_NAME - return TABLE_NAME; - case 5: // TABLE_TYPES - return TABLE_TYPES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.TABLE_TYPES}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); - tmpMap.put(_Fields.TABLE_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tableTypes", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesReq.class, metaDataMap); - } - - public TGetTablesReq() { - } - - public TGetTablesReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTablesReq(TGetTablesReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetCatalogName()) { - this.catalogName = other.catalogName; - } - if (other.isSetSchemaName()) { - this.schemaName = other.schemaName; - } - if (other.isSetTableName()) { - this.tableName = other.tableName; - } - if (other.isSetTableTypes()) { - List __this__tableTypes = new ArrayList(); - for (String other_element : other.tableTypes) { - __this__tableTypes.add(other_element); - } - this.tableTypes = __this__tableTypes; - } - } - - public TGetTablesReq deepCopy() { - return new TGetTablesReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.catalogName = null; - this.schemaName = null; - this.tableName = null; - this.tableTypes = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getCatalogName() { - return this.catalogName; - } - - public void setCatalogName(String catalogName) { - this.catalogName = catalogName; - } - - public void unsetCatalogName() { - this.catalogName = null; - } - - /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ - public boolean isSetCatalogName() { - return this.catalogName != null; - } - - public void setCatalogNameIsSet(boolean value) { - if (!value) { - this.catalogName = null; - } - } - - public String getSchemaName() { - return this.schemaName; - } - - public void setSchemaName(String schemaName) { - this.schemaName = schemaName; - } - - public void unsetSchemaName() { - this.schemaName = null; - } - - /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ - public boolean isSetSchemaName() { - return this.schemaName != null; - } - - public void setSchemaNameIsSet(boolean value) { - if (!value) { - this.schemaName = null; - } - } - - public String getTableName() { - return this.tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void unsetTableName() { - this.tableName = null; - } - - /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ - public boolean isSetTableName() { - return this.tableName != null; - } - - public void setTableNameIsSet(boolean value) { - if (!value) { - this.tableName = null; - } - } - - public int getTableTypesSize() { - return (this.tableTypes == null) ? 0 : this.tableTypes.size(); - } - - public java.util.Iterator getTableTypesIterator() { - return (this.tableTypes == null) ? null : this.tableTypes.iterator(); - } - - public void addToTableTypes(String elem) { - if (this.tableTypes == null) { - this.tableTypes = new ArrayList(); - } - this.tableTypes.add(elem); - } - - public List getTableTypes() { - return this.tableTypes; - } - - public void setTableTypes(List tableTypes) { - this.tableTypes = tableTypes; - } - - public void unsetTableTypes() { - this.tableTypes = null; - } - - /** Returns true if field tableTypes is set (has been assigned a value) and false otherwise */ - public boolean isSetTableTypes() { - return this.tableTypes != null; - } - - public void setTableTypesIsSet(boolean value) { - if (!value) { - this.tableTypes = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CATALOG_NAME: - if (value == null) { - unsetCatalogName(); - } else { - setCatalogName((String)value); - } - break; - - case SCHEMA_NAME: - if (value == null) { - unsetSchemaName(); - } else { - setSchemaName((String)value); - } - break; - - case TABLE_NAME: - if (value == null) { - unsetTableName(); - } else { - setTableName((String)value); - } - break; - - case TABLE_TYPES: - if (value == null) { - unsetTableTypes(); - } else { - setTableTypes((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case CATALOG_NAME: - return getCatalogName(); - - case SCHEMA_NAME: - return getSchemaName(); - - case TABLE_NAME: - return getTableName(); - - case TABLE_TYPES: - return getTableTypes(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case CATALOG_NAME: - return isSetCatalogName(); - case SCHEMA_NAME: - return isSetSchemaName(); - case TABLE_NAME: - return isSetTableName(); - case TABLE_TYPES: - return isSetTableTypes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTablesReq) - return this.equals((TGetTablesReq)that); - return false; - } - - public boolean equals(TGetTablesReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_catalogName = true && this.isSetCatalogName(); - boolean that_present_catalogName = true && that.isSetCatalogName(); - if (this_present_catalogName || that_present_catalogName) { - if (!(this_present_catalogName && that_present_catalogName)) - return false; - if (!this.catalogName.equals(that.catalogName)) - return false; - } - - boolean this_present_schemaName = true && this.isSetSchemaName(); - boolean that_present_schemaName = true && that.isSetSchemaName(); - if (this_present_schemaName || that_present_schemaName) { - if (!(this_present_schemaName && that_present_schemaName)) - return false; - if (!this.schemaName.equals(that.schemaName)) - return false; - } - - boolean this_present_tableName = true && this.isSetTableName(); - boolean that_present_tableName = true && that.isSetTableName(); - if (this_present_tableName || that_present_tableName) { - if (!(this_present_tableName && that_present_tableName)) - return false; - if (!this.tableName.equals(that.tableName)) - return false; - } - - boolean this_present_tableTypes = true && this.isSetTableTypes(); - boolean that_present_tableTypes = true && that.isSetTableTypes(); - if (this_present_tableTypes || that_present_tableTypes) { - if (!(this_present_tableTypes && that_present_tableTypes)) - return false; - if (!this.tableTypes.equals(that.tableTypes)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_catalogName = true && (isSetCatalogName()); - builder.append(present_catalogName); - if (present_catalogName) - builder.append(catalogName); - - boolean present_schemaName = true && (isSetSchemaName()); - builder.append(present_schemaName); - if (present_schemaName) - builder.append(schemaName); - - boolean present_tableName = true && (isSetTableName()); - builder.append(present_tableName); - if (present_tableName) - builder.append(tableName); - - boolean present_tableTypes = true && (isSetTableTypes()); - builder.append(present_tableTypes); - if (present_tableTypes) - builder.append(tableTypes); - - return builder.toHashCode(); - } - - public int compareTo(TGetTablesReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTablesReq typedOther = (TGetTablesReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetCatalogName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSchemaName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTableTypes()).compareTo(typedOther.isSetTableTypes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTableTypes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableTypes, typedOther.tableTypes); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTablesReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (isSetCatalogName()) { - if (!first) sb.append(", "); - sb.append("catalogName:"); - if (this.catalogName == null) { - sb.append("null"); - } else { - sb.append(this.catalogName); - } - first = false; - } - if (isSetSchemaName()) { - if (!first) sb.append(", "); - sb.append("schemaName:"); - if (this.schemaName == null) { - sb.append("null"); - } else { - sb.append(this.schemaName); - } - first = false; - } - if (isSetTableName()) { - if (!first) sb.append(", "); - sb.append("tableName:"); - if (this.tableName == null) { - sb.append("null"); - } else { - sb.append(this.tableName); - } - first = false; - } - if (isSetTableTypes()) { - if (!first) sb.append(", "); - sb.append("tableTypes:"); - if (this.tableTypes == null) { - sb.append("null"); - } else { - sb.append(this.tableTypes); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTablesReqStandardSchemeFactory implements SchemeFactory { - public TGetTablesReqStandardScheme getScheme() { - return new TGetTablesReqStandardScheme(); - } - } - - private static class TGetTablesReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // CATALOG_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SCHEMA_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // TABLE_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // TABLE_TYPES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list172 = iprot.readListBegin(); - struct.tableTypes = new ArrayList(_list172.size); - for (int _i173 = 0; _i173 < _list172.size; ++_i173) - { - String _elem174; // optional - _elem174 = iprot.readString(); - struct.tableTypes.add(_elem174); - } - iprot.readListEnd(); - } - struct.setTableTypesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.catalogName != null) { - if (struct.isSetCatalogName()) { - oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); - oprot.writeString(struct.catalogName); - oprot.writeFieldEnd(); - } - } - if (struct.schemaName != null) { - if (struct.isSetSchemaName()) { - oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); - oprot.writeString(struct.schemaName); - oprot.writeFieldEnd(); - } - } - if (struct.tableName != null) { - if (struct.isSetTableName()) { - oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); - oprot.writeString(struct.tableName); - oprot.writeFieldEnd(); - } - } - if (struct.tableTypes != null) { - if (struct.isSetTableTypes()) { - oprot.writeFieldBegin(TABLE_TYPES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableTypes.size())); - for (String _iter175 : struct.tableTypes) - { - oprot.writeString(_iter175); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTablesReqTupleSchemeFactory implements SchemeFactory { - public TGetTablesReqTupleScheme getScheme() { - return new TGetTablesReqTupleScheme(); - } - } - - private static class TGetTablesReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetCatalogName()) { - optionals.set(0); - } - if (struct.isSetSchemaName()) { - optionals.set(1); - } - if (struct.isSetTableName()) { - optionals.set(2); - } - if (struct.isSetTableTypes()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetCatalogName()) { - oprot.writeString(struct.catalogName); - } - if (struct.isSetSchemaName()) { - oprot.writeString(struct.schemaName); - } - if (struct.isSetTableName()) { - oprot.writeString(struct.tableName); - } - if (struct.isSetTableTypes()) { - { - oprot.writeI32(struct.tableTypes.size()); - for (String _iter176 : struct.tableTypes) - { - oprot.writeString(_iter176); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - struct.catalogName = iprot.readString(); - struct.setCatalogNameIsSet(true); - } - if (incoming.get(1)) { - struct.schemaName = iprot.readString(); - struct.setSchemaNameIsSet(true); - } - if (incoming.get(2)) { - struct.tableName = iprot.readString(); - struct.setTableNameIsSet(true); - } - if (incoming.get(3)) { - { - org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.tableTypes = new ArrayList(_list177.size); - for (int _i178 = 0; _i178 < _list177.size; ++_i178) - { - String _elem179; // optional - _elem179 = iprot.readString(); - struct.tableTypes.add(_elem179); - } - } - struct.setTableTypesIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java deleted file mode 100644 index d526f4478a24e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTablesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTablesRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTablesRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesResp.class, metaDataMap); - } - - public TGetTablesResp() { - } - - public TGetTablesResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTablesResp(TGetTablesResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTablesResp deepCopy() { - return new TGetTablesResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTablesResp) - return this.equals((TGetTablesResp)that); - return false; - } - - public boolean equals(TGetTablesResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTablesResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTablesResp typedOther = (TGetTablesResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTablesResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTablesRespStandardSchemeFactory implements SchemeFactory { - public TGetTablesRespStandardScheme getScheme() { - return new TGetTablesRespStandardScheme(); - } - } - - private static class TGetTablesRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTablesRespTupleSchemeFactory implements SchemeFactory { - public TGetTablesRespTupleScheme getScheme() { - return new TGetTablesRespTupleScheme(); - } - } - - private static class TGetTablesRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java deleted file mode 100644 index d40115e83ec45..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTypeInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTypeInfoReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTypeInfoReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoReq.class, metaDataMap); - } - - public TGetTypeInfoReq() { - } - - public TGetTypeInfoReq( - TSessionHandle sessionHandle) - { - this(); - this.sessionHandle = sessionHandle; - } - - /** - * Performs a deep copy on other. - */ - public TGetTypeInfoReq(TGetTypeInfoReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - } - - public TGetTypeInfoReq deepCopy() { - return new TGetTypeInfoReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTypeInfoReq) - return this.equals((TGetTypeInfoReq)that); - return false; - } - - public boolean equals(TGetTypeInfoReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTypeInfoReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTypeInfoReq typedOther = (TGetTypeInfoReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTypeInfoReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTypeInfoReqStandardSchemeFactory implements SchemeFactory { - public TGetTypeInfoReqStandardScheme getScheme() { - return new TGetTypeInfoReqStandardScheme(); - } - } - - private static class TGetTypeInfoReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTypeInfoReqTupleSchemeFactory implements SchemeFactory { - public TGetTypeInfoReqTupleScheme getScheme() { - return new TGetTypeInfoReqTupleScheme(); - } - } - - private static class TGetTypeInfoReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java deleted file mode 100644 index 59be1a33b55e2..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java +++ /dev/null @@ -1,505 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TGetTypeInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TGetTypeInfoRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TGetTypeInfoRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TOperationHandle operationHandle; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - OPERATION_HANDLE((short)2, "operationHandle"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // OPERATION_HANDLE - return OPERATION_HANDLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoResp.class, metaDataMap); - } - - public TGetTypeInfoResp() { - } - - public TGetTypeInfoResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TGetTypeInfoResp(TGetTypeInfoResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetOperationHandle()) { - this.operationHandle = new TOperationHandle(other.operationHandle); - } - } - - public TGetTypeInfoResp deepCopy() { - return new TGetTypeInfoResp(this); - } - - @Override - public void clear() { - this.status = null; - this.operationHandle = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public TOperationHandle getOperationHandle() { - return this.operationHandle; - } - - public void setOperationHandle(TOperationHandle operationHandle) { - this.operationHandle = operationHandle; - } - - public void unsetOperationHandle() { - this.operationHandle = null; - } - - /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationHandle() { - return this.operationHandle != null; - } - - public void setOperationHandleIsSet(boolean value) { - if (!value) { - this.operationHandle = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case OPERATION_HANDLE: - if (value == null) { - unsetOperationHandle(); - } else { - setOperationHandle((TOperationHandle)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case OPERATION_HANDLE: - return getOperationHandle(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case OPERATION_HANDLE: - return isSetOperationHandle(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TGetTypeInfoResp) - return this.equals((TGetTypeInfoResp)that); - return false; - } - - public boolean equals(TGetTypeInfoResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_operationHandle = true && this.isSetOperationHandle(); - boolean that_present_operationHandle = true && that.isSetOperationHandle(); - if (this_present_operationHandle || that_present_operationHandle) { - if (!(this_present_operationHandle && that_present_operationHandle)) - return false; - if (!this.operationHandle.equals(that.operationHandle)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_operationHandle = true && (isSetOperationHandle()); - builder.append(present_operationHandle); - if (present_operationHandle) - builder.append(operationHandle); - - return builder.toHashCode(); - } - - public int compareTo(TGetTypeInfoResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TGetTypeInfoResp typedOther = (TGetTypeInfoResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TGetTypeInfoResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (isSetOperationHandle()) { - if (!first) sb.append(", "); - sb.append("operationHandle:"); - if (this.operationHandle == null) { - sb.append("null"); - } else { - sb.append(this.operationHandle); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (operationHandle != null) { - operationHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TGetTypeInfoRespStandardSchemeFactory implements SchemeFactory { - public TGetTypeInfoRespStandardScheme getScheme() { - return new TGetTypeInfoRespStandardScheme(); - } - } - - private static class TGetTypeInfoRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationHandle != null) { - if (struct.isSetOperationHandle()) { - oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); - struct.operationHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TGetTypeInfoRespTupleSchemeFactory implements SchemeFactory { - public TGetTypeInfoRespTupleScheme getScheme() { - return new TGetTypeInfoRespTupleScheme(); - } - } - - private static class TGetTypeInfoRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - BitSet optionals = new BitSet(); - if (struct.isSetOperationHandle()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetOperationHandle()) { - struct.operationHandle.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.operationHandle = new TOperationHandle(); - struct.operationHandle.read(iprot); - struct.setOperationHandleIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java deleted file mode 100644 index 368273c341c7b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java +++ /dev/null @@ -1,506 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class THandleIdentifier implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THandleIdentifier"); - - private static final org.apache.thrift.protocol.TField GUID_FIELD_DESC = new org.apache.thrift.protocol.TField("guid", org.apache.thrift.protocol.TType.STRING, (short)1); - private static final org.apache.thrift.protocol.TField SECRET_FIELD_DESC = new org.apache.thrift.protocol.TField("secret", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new THandleIdentifierStandardSchemeFactory()); - schemes.put(TupleScheme.class, new THandleIdentifierTupleSchemeFactory()); - } - - private ByteBuffer guid; // required - private ByteBuffer secret; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - GUID((short)1, "guid"), - SECRET((short)2, "secret"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // GUID - return GUID; - case 2: // SECRET - return SECRET; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.GUID, new org.apache.thrift.meta_data.FieldMetaData("guid", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - tmpMap.put(_Fields.SECRET, new org.apache.thrift.meta_data.FieldMetaData("secret", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(THandleIdentifier.class, metaDataMap); - } - - public THandleIdentifier() { - } - - public THandleIdentifier( - ByteBuffer guid, - ByteBuffer secret) - { - this(); - this.guid = guid; - this.secret = secret; - } - - /** - * Performs a deep copy on other. - */ - public THandleIdentifier(THandleIdentifier other) { - if (other.isSetGuid()) { - this.guid = org.apache.thrift.TBaseHelper.copyBinary(other.guid); -; - } - if (other.isSetSecret()) { - this.secret = org.apache.thrift.TBaseHelper.copyBinary(other.secret); -; - } - } - - public THandleIdentifier deepCopy() { - return new THandleIdentifier(this); - } - - @Override - public void clear() { - this.guid = null; - this.secret = null; - } - - public byte[] getGuid() { - setGuid(org.apache.thrift.TBaseHelper.rightSize(guid)); - return guid == null ? null : guid.array(); - } - - public ByteBuffer bufferForGuid() { - return guid; - } - - public void setGuid(byte[] guid) { - setGuid(guid == null ? (ByteBuffer)null : ByteBuffer.wrap(guid)); - } - - public void setGuid(ByteBuffer guid) { - this.guid = guid; - } - - public void unsetGuid() { - this.guid = null; - } - - /** Returns true if field guid is set (has been assigned a value) and false otherwise */ - public boolean isSetGuid() { - return this.guid != null; - } - - public void setGuidIsSet(boolean value) { - if (!value) { - this.guid = null; - } - } - - public byte[] getSecret() { - setSecret(org.apache.thrift.TBaseHelper.rightSize(secret)); - return secret == null ? null : secret.array(); - } - - public ByteBuffer bufferForSecret() { - return secret; - } - - public void setSecret(byte[] secret) { - setSecret(secret == null ? (ByteBuffer)null : ByteBuffer.wrap(secret)); - } - - public void setSecret(ByteBuffer secret) { - this.secret = secret; - } - - public void unsetSecret() { - this.secret = null; - } - - /** Returns true if field secret is set (has been assigned a value) and false otherwise */ - public boolean isSetSecret() { - return this.secret != null; - } - - public void setSecretIsSet(boolean value) { - if (!value) { - this.secret = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case GUID: - if (value == null) { - unsetGuid(); - } else { - setGuid((ByteBuffer)value); - } - break; - - case SECRET: - if (value == null) { - unsetSecret(); - } else { - setSecret((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case GUID: - return getGuid(); - - case SECRET: - return getSecret(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case GUID: - return isSetGuid(); - case SECRET: - return isSetSecret(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof THandleIdentifier) - return this.equals((THandleIdentifier)that); - return false; - } - - public boolean equals(THandleIdentifier that) { - if (that == null) - return false; - - boolean this_present_guid = true && this.isSetGuid(); - boolean that_present_guid = true && that.isSetGuid(); - if (this_present_guid || that_present_guid) { - if (!(this_present_guid && that_present_guid)) - return false; - if (!this.guid.equals(that.guid)) - return false; - } - - boolean this_present_secret = true && this.isSetSecret(); - boolean that_present_secret = true && that.isSetSecret(); - if (this_present_secret || that_present_secret) { - if (!(this_present_secret && that_present_secret)) - return false; - if (!this.secret.equals(that.secret)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_guid = true && (isSetGuid()); - builder.append(present_guid); - if (present_guid) - builder.append(guid); - - boolean present_secret = true && (isSetSecret()); - builder.append(present_secret); - if (present_secret) - builder.append(secret); - - return builder.toHashCode(); - } - - public int compareTo(THandleIdentifier other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - THandleIdentifier typedOther = (THandleIdentifier)other; - - lastComparison = Boolean.valueOf(isSetGuid()).compareTo(typedOther.isSetGuid()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetGuid()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.guid, typedOther.guid); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSecret()).compareTo(typedOther.isSetSecret()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSecret()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.secret, typedOther.secret); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("THandleIdentifier("); - boolean first = true; - - sb.append("guid:"); - if (this.guid == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.guid, sb); - } - first = false; - if (!first) sb.append(", "); - sb.append("secret:"); - if (this.secret == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.secret, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetGuid()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'guid' is unset! Struct:" + toString()); - } - - if (!isSetSecret()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'secret' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class THandleIdentifierStandardSchemeFactory implements SchemeFactory { - public THandleIdentifierStandardScheme getScheme() { - return new THandleIdentifierStandardScheme(); - } - } - - private static class THandleIdentifierStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, THandleIdentifier struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // GUID - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.guid = iprot.readBinary(); - struct.setGuidIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SECRET - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.secret = iprot.readBinary(); - struct.setSecretIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, THandleIdentifier struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.guid != null) { - oprot.writeFieldBegin(GUID_FIELD_DESC); - oprot.writeBinary(struct.guid); - oprot.writeFieldEnd(); - } - if (struct.secret != null) { - oprot.writeFieldBegin(SECRET_FIELD_DESC); - oprot.writeBinary(struct.secret); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class THandleIdentifierTupleSchemeFactory implements SchemeFactory { - public THandleIdentifierTupleScheme getScheme() { - return new THandleIdentifierTupleScheme(); - } - } - - private static class THandleIdentifierTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeBinary(struct.guid); - oprot.writeBinary(struct.secret); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.guid = iprot.readBinary(); - struct.setGuidIsSet(true); - struct.secret = iprot.readBinary(); - struct.setSecretIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java deleted file mode 100644 index c83663072f877..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI16Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI16ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI16ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Column.class, metaDataMap); - } - - public TI16Column() { - } - - public TI16Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI16Column(TI16Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Short other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI16Column deepCopy() { - return new TI16Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(short elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI16Column) - return this.equals((TI16Column)that); - return false; - } - - public boolean equals(TI16Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI16Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI16Column typedOther = (TI16Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI16Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI16ColumnStandardSchemeFactory implements SchemeFactory { - public TI16ColumnStandardScheme getScheme() { - return new TI16ColumnStandardScheme(); - } - } - - private static class TI16ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); - struct.values = new ArrayList(_list70.size); - for (int _i71 = 0; _i71 < _list70.size; ++_i71) - { - short _elem72; // optional - _elem72 = iprot.readI16(); - struct.values.add(_elem72); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, struct.values.size())); - for (short _iter73 : struct.values) - { - oprot.writeI16(_iter73); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI16ColumnTupleSchemeFactory implements SchemeFactory { - public TI16ColumnTupleScheme getScheme() { - return new TI16ColumnTupleScheme(); - } - } - - private static class TI16ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (short _iter74 : struct.values) - { - oprot.writeI16(_iter74); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list75 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, iprot.readI32()); - struct.values = new ArrayList(_list75.size); - for (int _i76 = 0; _i76 < _list75.size; ++_i76) - { - short _elem77; // optional - _elem77 = iprot.readI16(); - struct.values.add(_elem77); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java deleted file mode 100644 index bb5ae9609de86..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI16Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I16, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI16ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI16ValueTupleSchemeFactory()); - } - - private short value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Value.class, metaDataMap); - } - - public TI16Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI16Value(TI16Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI16Value deepCopy() { - return new TI16Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public short getValue() { - return this.value; - } - - public void setValue(short value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Short)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Short.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI16Value) - return this.equals((TI16Value)that); - return false; - } - - public boolean equals(TI16Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI16Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI16Value typedOther = (TI16Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI16Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI16ValueStandardSchemeFactory implements SchemeFactory { - public TI16ValueStandardScheme getScheme() { - return new TI16ValueStandardScheme(); - } - } - - private static class TI16ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I16) { - struct.value = iprot.readI16(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI16(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI16ValueTupleSchemeFactory implements SchemeFactory { - public TI16ValueTupleScheme getScheme() { - return new TI16ValueTupleScheme(); - } - } - - private static class TI16ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI16(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI16(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java deleted file mode 100644 index 6c6c5f35b7c8e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI32Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI32ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI32ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Column.class, metaDataMap); - } - - public TI32Column() { - } - - public TI32Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI32Column(TI32Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Integer other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI32Column deepCopy() { - return new TI32Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(int elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI32Column) - return this.equals((TI32Column)that); - return false; - } - - public boolean equals(TI32Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI32Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI32Column typedOther = (TI32Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI32Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI32ColumnStandardSchemeFactory implements SchemeFactory { - public TI32ColumnStandardScheme getScheme() { - return new TI32ColumnStandardScheme(); - } - } - - private static class TI32ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); - struct.values = new ArrayList(_list78.size); - for (int _i79 = 0; _i79 < _list78.size; ++_i79) - { - int _elem80; // optional - _elem80 = iprot.readI32(); - struct.values.add(_elem80); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size())); - for (int _iter81 : struct.values) - { - oprot.writeI32(_iter81); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI32ColumnTupleSchemeFactory implements SchemeFactory { - public TI32ColumnTupleScheme getScheme() { - return new TI32ColumnTupleScheme(); - } - } - - private static class TI32ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (int _iter82 : struct.values) - { - oprot.writeI32(_iter82); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list83 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.values = new ArrayList(_list83.size); - for (int _i84 = 0; _i84 < _list83.size; ++_i84) - { - int _elem85; // optional - _elem85 = iprot.readI32(); - struct.values.add(_elem85); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java deleted file mode 100644 index 059408b96c8ce..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI32Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I32, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI32ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI32ValueTupleSchemeFactory()); - } - - private int value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Value.class, metaDataMap); - } - - public TI32Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI32Value(TI32Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI32Value deepCopy() { - return new TI32Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public int getValue() { - return this.value; - } - - public void setValue(int value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Integer.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI32Value) - return this.equals((TI32Value)that); - return false; - } - - public boolean equals(TI32Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI32Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI32Value typedOther = (TI32Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI32Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI32ValueStandardSchemeFactory implements SchemeFactory { - public TI32ValueStandardScheme getScheme() { - return new TI32ValueStandardScheme(); - } - } - - private static class TI32ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.value = iprot.readI32(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI32(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI32ValueTupleSchemeFactory implements SchemeFactory { - public TI32ValueTupleScheme getScheme() { - return new TI32ValueTupleScheme(); - } - } - - private static class TI32ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI32(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI32(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java deleted file mode 100644 index cc383ed089fa4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI64Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Column"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI64ColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI64ColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Column.class, metaDataMap); - } - - public TI64Column() { - } - - public TI64Column( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TI64Column(TI64Column other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (Long other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TI64Column deepCopy() { - return new TI64Column(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(long elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI64Column) - return this.equals((TI64Column)that); - return false; - } - - public boolean equals(TI64Column that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TI64Column other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI64Column typedOther = (TI64Column)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI64Column("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI64ColumnStandardSchemeFactory implements SchemeFactory { - public TI64ColumnStandardScheme getScheme() { - return new TI64ColumnStandardScheme(); - } - } - - private static class TI64ColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Column struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list86 = iprot.readListBegin(); - struct.values = new ArrayList(_list86.size); - for (int _i87 = 0; _i87 < _list86.size; ++_i87) - { - long _elem88; // optional - _elem88 = iprot.readI64(); - struct.values.add(_elem88); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Column struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.values.size())); - for (long _iter89 : struct.values) - { - oprot.writeI64(_iter89); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI64ColumnTupleSchemeFactory implements SchemeFactory { - public TI64ColumnTupleScheme getScheme() { - return new TI64ColumnTupleScheme(); - } - } - - private static class TI64ColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (long _iter90 : struct.values) - { - oprot.writeI64(_iter90); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list91 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.values = new ArrayList(_list91.size); - for (int _i92 = 0; _i92 < _list91.size; ++_i92) - { - long _elem93; // optional - _elem93 = iprot.readI64(); - struct.values.add(_elem93); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java deleted file mode 100644 index 9a941cce0c077..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java +++ /dev/null @@ -1,386 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TI64Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Value"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I64, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TI64ValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TI64ValueTupleSchemeFactory()); - } - - private long value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Value.class, metaDataMap); - } - - public TI64Value() { - } - - /** - * Performs a deep copy on other. - */ - public TI64Value(TI64Value other) { - __isset_bitfield = other.__isset_bitfield; - this.value = other.value; - } - - public TI64Value deepCopy() { - return new TI64Value(this); - } - - @Override - public void clear() { - setValueIsSet(false); - this.value = 0; - } - - public long getValue() { - return this.value; - } - - public void setValue(long value) { - this.value = value; - setValueIsSet(true); - } - - public void unsetValue() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); - } - - public void setValueIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((Long)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return Long.valueOf(getValue()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TI64Value) - return this.equals((TI64Value)that); - return false; - } - - public boolean equals(TI64Value that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TI64Value other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TI64Value typedOther = (TI64Value)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TI64Value("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - sb.append(this.value); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TI64ValueStandardSchemeFactory implements SchemeFactory { - public TI64ValueStandardScheme getScheme() { - return new TI64ValueStandardScheme(); - } - } - - private static class TI64ValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Value struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.value = iprot.readI64(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Value struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeI64(struct.value); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TI64ValueTupleSchemeFactory implements SchemeFactory { - public TI64ValueTupleScheme getScheme() { - return new TI64ValueTupleScheme(); - } - } - - private static class TI64ValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeI64(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readI64(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java deleted file mode 100644 index 425603cbdecbd..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java +++ /dev/null @@ -1,478 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TMapTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMapTypeEntry"); - - private static final org.apache.thrift.protocol.TField KEY_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("keyTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField VALUE_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("valueTypePtr", org.apache.thrift.protocol.TType.I32, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TMapTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TMapTypeEntryTupleSchemeFactory()); - } - - private int keyTypePtr; // required - private int valueTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - KEY_TYPE_PTR((short)1, "keyTypePtr"), - VALUE_TYPE_PTR((short)2, "valueTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // KEY_TYPE_PTR - return KEY_TYPE_PTR; - case 2: // VALUE_TYPE_PTR - return VALUE_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __KEYTYPEPTR_ISSET_ID = 0; - private static final int __VALUETYPEPTR_ISSET_ID = 1; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.KEY_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("keyTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - tmpMap.put(_Fields.VALUE_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("valueTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TMapTypeEntry.class, metaDataMap); - } - - public TMapTypeEntry() { - } - - public TMapTypeEntry( - int keyTypePtr, - int valueTypePtr) - { - this(); - this.keyTypePtr = keyTypePtr; - setKeyTypePtrIsSet(true); - this.valueTypePtr = valueTypePtr; - setValueTypePtrIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TMapTypeEntry(TMapTypeEntry other) { - __isset_bitfield = other.__isset_bitfield; - this.keyTypePtr = other.keyTypePtr; - this.valueTypePtr = other.valueTypePtr; - } - - public TMapTypeEntry deepCopy() { - return new TMapTypeEntry(this); - } - - @Override - public void clear() { - setKeyTypePtrIsSet(false); - this.keyTypePtr = 0; - setValueTypePtrIsSet(false); - this.valueTypePtr = 0; - } - - public int getKeyTypePtr() { - return this.keyTypePtr; - } - - public void setKeyTypePtr(int keyTypePtr) { - this.keyTypePtr = keyTypePtr; - setKeyTypePtrIsSet(true); - } - - public void unsetKeyTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); - } - - /** Returns true if field keyTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetKeyTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); - } - - public void setKeyTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID, value); - } - - public int getValueTypePtr() { - return this.valueTypePtr; - } - - public void setValueTypePtr(int valueTypePtr) { - this.valueTypePtr = valueTypePtr; - setValueTypePtrIsSet(true); - } - - public void unsetValueTypePtr() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); - } - - /** Returns true if field valueTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetValueTypePtr() { - return EncodingUtils.testBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); - } - - public void setValueTypePtrIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case KEY_TYPE_PTR: - if (value == null) { - unsetKeyTypePtr(); - } else { - setKeyTypePtr((Integer)value); - } - break; - - case VALUE_TYPE_PTR: - if (value == null) { - unsetValueTypePtr(); - } else { - setValueTypePtr((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case KEY_TYPE_PTR: - return Integer.valueOf(getKeyTypePtr()); - - case VALUE_TYPE_PTR: - return Integer.valueOf(getValueTypePtr()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case KEY_TYPE_PTR: - return isSetKeyTypePtr(); - case VALUE_TYPE_PTR: - return isSetValueTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TMapTypeEntry) - return this.equals((TMapTypeEntry)that); - return false; - } - - public boolean equals(TMapTypeEntry that) { - if (that == null) - return false; - - boolean this_present_keyTypePtr = true; - boolean that_present_keyTypePtr = true; - if (this_present_keyTypePtr || that_present_keyTypePtr) { - if (!(this_present_keyTypePtr && that_present_keyTypePtr)) - return false; - if (this.keyTypePtr != that.keyTypePtr) - return false; - } - - boolean this_present_valueTypePtr = true; - boolean that_present_valueTypePtr = true; - if (this_present_valueTypePtr || that_present_valueTypePtr) { - if (!(this_present_valueTypePtr && that_present_valueTypePtr)) - return false; - if (this.valueTypePtr != that.valueTypePtr) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_keyTypePtr = true; - builder.append(present_keyTypePtr); - if (present_keyTypePtr) - builder.append(keyTypePtr); - - boolean present_valueTypePtr = true; - builder.append(present_valueTypePtr); - if (present_valueTypePtr) - builder.append(valueTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TMapTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TMapTypeEntry typedOther = (TMapTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetKeyTypePtr()).compareTo(typedOther.isSetKeyTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetKeyTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyTypePtr, typedOther.keyTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetValueTypePtr()).compareTo(typedOther.isSetValueTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValueTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.valueTypePtr, typedOther.valueTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TMapTypeEntry("); - boolean first = true; - - sb.append("keyTypePtr:"); - sb.append(this.keyTypePtr); - first = false; - if (!first) sb.append(", "); - sb.append("valueTypePtr:"); - sb.append(this.valueTypePtr); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetKeyTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'keyTypePtr' is unset! Struct:" + toString()); - } - - if (!isSetValueTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'valueTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TMapTypeEntryStandardSchemeFactory implements SchemeFactory { - public TMapTypeEntryStandardScheme getScheme() { - return new TMapTypeEntryStandardScheme(); - } - } - - private static class TMapTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TMapTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // KEY_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.keyTypePtr = iprot.readI32(); - struct.setKeyTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // VALUE_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.valueTypePtr = iprot.readI32(); - struct.setValueTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TMapTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(KEY_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.keyTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(VALUE_TYPE_PTR_FIELD_DESC); - oprot.writeI32(struct.valueTypePtr); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TMapTypeEntryTupleSchemeFactory implements SchemeFactory { - public TMapTypeEntryTupleScheme getScheme() { - return new TMapTypeEntryTupleScheme(); - } - } - - private static class TMapTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.keyTypePtr); - oprot.writeI32(struct.valueTypePtr); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.keyTypePtr = iprot.readI32(); - struct.setKeyTypePtrIsSet(true); - struct.valueTypePtr = iprot.readI32(); - struct.setValueTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java deleted file mode 100644 index c0481615b06d3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java +++ /dev/null @@ -1,785 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOpenSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionReq"); - - private static final org.apache.thrift.protocol.TField CLIENT_PROTOCOL_FIELD_DESC = new org.apache.thrift.protocol.TField("client_protocol", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField USERNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("username", org.apache.thrift.protocol.TType.STRING, (short)2); - private static final org.apache.thrift.protocol.TField PASSWORD_FIELD_DESC = new org.apache.thrift.protocol.TField("password", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOpenSessionReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOpenSessionReqTupleSchemeFactory()); - } - - private TProtocolVersion client_protocol; // required - private String username; // optional - private String password; // optional - private Map configuration; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TProtocolVersion - */ - CLIENT_PROTOCOL((short)1, "client_protocol"), - USERNAME((short)2, "username"), - PASSWORD((short)3, "password"), - CONFIGURATION((short)4, "configuration"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // CLIENT_PROTOCOL - return CLIENT_PROTOCOL; - case 2: // USERNAME - return USERNAME; - case 3: // PASSWORD - return PASSWORD; - case 4: // CONFIGURATION - return CONFIGURATION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.USERNAME,_Fields.PASSWORD,_Fields.CONFIGURATION}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.CLIENT_PROTOCOL, new org.apache.thrift.meta_data.FieldMetaData("client_protocol", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); - tmpMap.put(_Fields.USERNAME, new org.apache.thrift.meta_data.FieldMetaData("username", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.PASSWORD, new org.apache.thrift.meta_data.FieldMetaData("password", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionReq.class, metaDataMap); - } - - public TOpenSessionReq() { - this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - } - - public TOpenSessionReq( - TProtocolVersion client_protocol) - { - this(); - this.client_protocol = client_protocol; - } - - /** - * Performs a deep copy on other. - */ - public TOpenSessionReq(TOpenSessionReq other) { - if (other.isSetClient_protocol()) { - this.client_protocol = other.client_protocol; - } - if (other.isSetUsername()) { - this.username = other.username; - } - if (other.isSetPassword()) { - this.password = other.password; - } - if (other.isSetConfiguration()) { - Map __this__configuration = new HashMap(); - for (Map.Entry other_element : other.configuration.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__configuration_copy_key = other_element_key; - - String __this__configuration_copy_value = other_element_value; - - __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); - } - this.configuration = __this__configuration; - } - } - - public TOpenSessionReq deepCopy() { - return new TOpenSessionReq(this); - } - - @Override - public void clear() { - this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - this.username = null; - this.password = null; - this.configuration = null; - } - - /** - * - * @see TProtocolVersion - */ - public TProtocolVersion getClient_protocol() { - return this.client_protocol; - } - - /** - * - * @see TProtocolVersion - */ - public void setClient_protocol(TProtocolVersion client_protocol) { - this.client_protocol = client_protocol; - } - - public void unsetClient_protocol() { - this.client_protocol = null; - } - - /** Returns true if field client_protocol is set (has been assigned a value) and false otherwise */ - public boolean isSetClient_protocol() { - return this.client_protocol != null; - } - - public void setClient_protocolIsSet(boolean value) { - if (!value) { - this.client_protocol = null; - } - } - - public String getUsername() { - return this.username; - } - - public void setUsername(String username) { - this.username = username; - } - - public void unsetUsername() { - this.username = null; - } - - /** Returns true if field username is set (has been assigned a value) and false otherwise */ - public boolean isSetUsername() { - return this.username != null; - } - - public void setUsernameIsSet(boolean value) { - if (!value) { - this.username = null; - } - } - - public String getPassword() { - return this.password; - } - - public void setPassword(String password) { - this.password = password; - } - - public void unsetPassword() { - this.password = null; - } - - /** Returns true if field password is set (has been assigned a value) and false otherwise */ - public boolean isSetPassword() { - return this.password != null; - } - - public void setPasswordIsSet(boolean value) { - if (!value) { - this.password = null; - } - } - - public int getConfigurationSize() { - return (this.configuration == null) ? 0 : this.configuration.size(); - } - - public void putToConfiguration(String key, String val) { - if (this.configuration == null) { - this.configuration = new HashMap(); - } - this.configuration.put(key, val); - } - - public Map getConfiguration() { - return this.configuration; - } - - public void setConfiguration(Map configuration) { - this.configuration = configuration; - } - - public void unsetConfiguration() { - this.configuration = null; - } - - /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ - public boolean isSetConfiguration() { - return this.configuration != null; - } - - public void setConfigurationIsSet(boolean value) { - if (!value) { - this.configuration = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case CLIENT_PROTOCOL: - if (value == null) { - unsetClient_protocol(); - } else { - setClient_protocol((TProtocolVersion)value); - } - break; - - case USERNAME: - if (value == null) { - unsetUsername(); - } else { - setUsername((String)value); - } - break; - - case PASSWORD: - if (value == null) { - unsetPassword(); - } else { - setPassword((String)value); - } - break; - - case CONFIGURATION: - if (value == null) { - unsetConfiguration(); - } else { - setConfiguration((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case CLIENT_PROTOCOL: - return getClient_protocol(); - - case USERNAME: - return getUsername(); - - case PASSWORD: - return getPassword(); - - case CONFIGURATION: - return getConfiguration(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case CLIENT_PROTOCOL: - return isSetClient_protocol(); - case USERNAME: - return isSetUsername(); - case PASSWORD: - return isSetPassword(); - case CONFIGURATION: - return isSetConfiguration(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOpenSessionReq) - return this.equals((TOpenSessionReq)that); - return false; - } - - public boolean equals(TOpenSessionReq that) { - if (that == null) - return false; - - boolean this_present_client_protocol = true && this.isSetClient_protocol(); - boolean that_present_client_protocol = true && that.isSetClient_protocol(); - if (this_present_client_protocol || that_present_client_protocol) { - if (!(this_present_client_protocol && that_present_client_protocol)) - return false; - if (!this.client_protocol.equals(that.client_protocol)) - return false; - } - - boolean this_present_username = true && this.isSetUsername(); - boolean that_present_username = true && that.isSetUsername(); - if (this_present_username || that_present_username) { - if (!(this_present_username && that_present_username)) - return false; - if (!this.username.equals(that.username)) - return false; - } - - boolean this_present_password = true && this.isSetPassword(); - boolean that_present_password = true && that.isSetPassword(); - if (this_present_password || that_present_password) { - if (!(this_present_password && that_present_password)) - return false; - if (!this.password.equals(that.password)) - return false; - } - - boolean this_present_configuration = true && this.isSetConfiguration(); - boolean that_present_configuration = true && that.isSetConfiguration(); - if (this_present_configuration || that_present_configuration) { - if (!(this_present_configuration && that_present_configuration)) - return false; - if (!this.configuration.equals(that.configuration)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_client_protocol = true && (isSetClient_protocol()); - builder.append(present_client_protocol); - if (present_client_protocol) - builder.append(client_protocol.getValue()); - - boolean present_username = true && (isSetUsername()); - builder.append(present_username); - if (present_username) - builder.append(username); - - boolean present_password = true && (isSetPassword()); - builder.append(present_password); - if (present_password) - builder.append(password); - - boolean present_configuration = true && (isSetConfiguration()); - builder.append(present_configuration); - if (present_configuration) - builder.append(configuration); - - return builder.toHashCode(); - } - - public int compareTo(TOpenSessionReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOpenSessionReq typedOther = (TOpenSessionReq)other; - - lastComparison = Boolean.valueOf(isSetClient_protocol()).compareTo(typedOther.isSetClient_protocol()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetClient_protocol()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.client_protocol, typedOther.client_protocol); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetUsername()).compareTo(typedOther.isSetUsername()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetUsername()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.username, typedOther.username); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPassword()).compareTo(typedOther.isSetPassword()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPassword()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.password, typedOther.password); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfiguration()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOpenSessionReq("); - boolean first = true; - - sb.append("client_protocol:"); - if (this.client_protocol == null) { - sb.append("null"); - } else { - sb.append(this.client_protocol); - } - first = false; - if (isSetUsername()) { - if (!first) sb.append(", "); - sb.append("username:"); - if (this.username == null) { - sb.append("null"); - } else { - sb.append(this.username); - } - first = false; - } - if (isSetPassword()) { - if (!first) sb.append(", "); - sb.append("password:"); - if (this.password == null) { - sb.append("null"); - } else { - sb.append(this.password); - } - first = false; - } - if (isSetConfiguration()) { - if (!first) sb.append(", "); - sb.append("configuration:"); - if (this.configuration == null) { - sb.append("null"); - } else { - sb.append(this.configuration); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetClient_protocol()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'client_protocol' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOpenSessionReqStandardSchemeFactory implements SchemeFactory { - public TOpenSessionReqStandardScheme getScheme() { - return new TOpenSessionReqStandardScheme(); - } - } - - private static class TOpenSessionReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // CLIENT_PROTOCOL - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); - struct.setClient_protocolIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // USERNAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.username = iprot.readString(); - struct.setUsernameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // PASSWORD - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.password = iprot.readString(); - struct.setPasswordIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // CONFIGURATION - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map142.size); - for (int _i143 = 0; _i143 < _map142.size; ++_i143) - { - String _key144; // required - String _val145; // required - _key144 = iprot.readString(); - _val145 = iprot.readString(); - struct.configuration.put(_key144, _val145); - } - iprot.readMapEnd(); - } - struct.setConfigurationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.client_protocol != null) { - oprot.writeFieldBegin(CLIENT_PROTOCOL_FIELD_DESC); - oprot.writeI32(struct.client_protocol.getValue()); - oprot.writeFieldEnd(); - } - if (struct.username != null) { - if (struct.isSetUsername()) { - oprot.writeFieldBegin(USERNAME_FIELD_DESC); - oprot.writeString(struct.username); - oprot.writeFieldEnd(); - } - } - if (struct.password != null) { - if (struct.isSetPassword()) { - oprot.writeFieldBegin(PASSWORD_FIELD_DESC); - oprot.writeString(struct.password); - oprot.writeFieldEnd(); - } - } - if (struct.configuration != null) { - if (struct.isSetConfiguration()) { - oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter146 : struct.configuration.entrySet()) - { - oprot.writeString(_iter146.getKey()); - oprot.writeString(_iter146.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOpenSessionReqTupleSchemeFactory implements SchemeFactory { - public TOpenSessionReqTupleScheme getScheme() { - return new TOpenSessionReqTupleScheme(); - } - } - - private static class TOpenSessionReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.client_protocol.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetUsername()) { - optionals.set(0); - } - if (struct.isSetPassword()) { - optionals.set(1); - } - if (struct.isSetConfiguration()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); - if (struct.isSetUsername()) { - oprot.writeString(struct.username); - } - if (struct.isSetPassword()) { - oprot.writeString(struct.password); - } - if (struct.isSetConfiguration()) { - { - oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter147 : struct.configuration.entrySet()) - { - oprot.writeString(_iter147.getKey()); - oprot.writeString(_iter147.getValue()); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); - struct.setClient_protocolIsSet(true); - BitSet incoming = iprot.readBitSet(3); - if (incoming.get(0)) { - struct.username = iprot.readString(); - struct.setUsernameIsSet(true); - } - if (incoming.get(1)) { - struct.password = iprot.readString(); - struct.setPasswordIsSet(true); - } - if (incoming.get(2)) { - { - org.apache.thrift.protocol.TMap _map148 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map148.size); - for (int _i149 = 0; _i149 < _map148.size; ++_i149) - { - String _key150; // required - String _val151; // required - _key150 = iprot.readString(); - _val151 = iprot.readString(); - struct.configuration.put(_key150, _val151); - } - } - struct.setConfigurationIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java deleted file mode 100644 index 351f78b2de20c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java +++ /dev/null @@ -1,790 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOpenSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField SERVER_PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("serverProtocolVersion", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOpenSessionRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOpenSessionRespTupleSchemeFactory()); - } - - private TStatus status; // required - private TProtocolVersion serverProtocolVersion; // required - private TSessionHandle sessionHandle; // optional - private Map configuration; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"), - /** - * - * @see TProtocolVersion - */ - SERVER_PROTOCOL_VERSION((short)2, "serverProtocolVersion"), - SESSION_HANDLE((short)3, "sessionHandle"), - CONFIGURATION((short)4, "configuration"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - case 2: // SERVER_PROTOCOL_VERSION - return SERVER_PROTOCOL_VERSION; - case 3: // SESSION_HANDLE - return SESSION_HANDLE; - case 4: // CONFIGURATION - return CONFIGURATION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.SESSION_HANDLE,_Fields.CONFIGURATION}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - tmpMap.put(_Fields.SERVER_PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("serverProtocolVersion", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionResp.class, metaDataMap); - } - - public TOpenSessionResp() { - this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - } - - public TOpenSessionResp( - TStatus status, - TProtocolVersion serverProtocolVersion) - { - this(); - this.status = status; - this.serverProtocolVersion = serverProtocolVersion; - } - - /** - * Performs a deep copy on other. - */ - public TOpenSessionResp(TOpenSessionResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - if (other.isSetServerProtocolVersion()) { - this.serverProtocolVersion = other.serverProtocolVersion; - } - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetConfiguration()) { - Map __this__configuration = new HashMap(); - for (Map.Entry other_element : other.configuration.entrySet()) { - - String other_element_key = other_element.getKey(); - String other_element_value = other_element.getValue(); - - String __this__configuration_copy_key = other_element_key; - - String __this__configuration_copy_value = other_element_value; - - __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); - } - this.configuration = __this__configuration; - } - } - - public TOpenSessionResp deepCopy() { - return new TOpenSessionResp(this); - } - - @Override - public void clear() { - this.status = null; - this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; - - this.sessionHandle = null; - this.configuration = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - /** - * - * @see TProtocolVersion - */ - public TProtocolVersion getServerProtocolVersion() { - return this.serverProtocolVersion; - } - - /** - * - * @see TProtocolVersion - */ - public void setServerProtocolVersion(TProtocolVersion serverProtocolVersion) { - this.serverProtocolVersion = serverProtocolVersion; - } - - public void unsetServerProtocolVersion() { - this.serverProtocolVersion = null; - } - - /** Returns true if field serverProtocolVersion is set (has been assigned a value) and false otherwise */ - public boolean isSetServerProtocolVersion() { - return this.serverProtocolVersion != null; - } - - public void setServerProtocolVersionIsSet(boolean value) { - if (!value) { - this.serverProtocolVersion = null; - } - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public int getConfigurationSize() { - return (this.configuration == null) ? 0 : this.configuration.size(); - } - - public void putToConfiguration(String key, String val) { - if (this.configuration == null) { - this.configuration = new HashMap(); - } - this.configuration.put(key, val); - } - - public Map getConfiguration() { - return this.configuration; - } - - public void setConfiguration(Map configuration) { - this.configuration = configuration; - } - - public void unsetConfiguration() { - this.configuration = null; - } - - /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ - public boolean isSetConfiguration() { - return this.configuration != null; - } - - public void setConfigurationIsSet(boolean value) { - if (!value) { - this.configuration = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - case SERVER_PROTOCOL_VERSION: - if (value == null) { - unsetServerProtocolVersion(); - } else { - setServerProtocolVersion((TProtocolVersion)value); - } - break; - - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case CONFIGURATION: - if (value == null) { - unsetConfiguration(); - } else { - setConfiguration((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - case SERVER_PROTOCOL_VERSION: - return getServerProtocolVersion(); - - case SESSION_HANDLE: - return getSessionHandle(); - - case CONFIGURATION: - return getConfiguration(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - case SERVER_PROTOCOL_VERSION: - return isSetServerProtocolVersion(); - case SESSION_HANDLE: - return isSetSessionHandle(); - case CONFIGURATION: - return isSetConfiguration(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOpenSessionResp) - return this.equals((TOpenSessionResp)that); - return false; - } - - public boolean equals(TOpenSessionResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_serverProtocolVersion = true && this.isSetServerProtocolVersion(); - boolean that_present_serverProtocolVersion = true && that.isSetServerProtocolVersion(); - if (this_present_serverProtocolVersion || that_present_serverProtocolVersion) { - if (!(this_present_serverProtocolVersion && that_present_serverProtocolVersion)) - return false; - if (!this.serverProtocolVersion.equals(that.serverProtocolVersion)) - return false; - } - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_configuration = true && this.isSetConfiguration(); - boolean that_present_configuration = true && that.isSetConfiguration(); - if (this_present_configuration || that_present_configuration) { - if (!(this_present_configuration && that_present_configuration)) - return false; - if (!this.configuration.equals(that.configuration)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_serverProtocolVersion = true && (isSetServerProtocolVersion()); - builder.append(present_serverProtocolVersion); - if (present_serverProtocolVersion) - builder.append(serverProtocolVersion.getValue()); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_configuration = true && (isSetConfiguration()); - builder.append(present_configuration); - if (present_configuration) - builder.append(configuration); - - return builder.toHashCode(); - } - - public int compareTo(TOpenSessionResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOpenSessionResp typedOther = (TOpenSessionResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetServerProtocolVersion()).compareTo(typedOther.isSetServerProtocolVersion()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetServerProtocolVersion()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverProtocolVersion, typedOther.serverProtocolVersion); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetConfiguration()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOpenSessionResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("serverProtocolVersion:"); - if (this.serverProtocolVersion == null) { - sb.append("null"); - } else { - sb.append(this.serverProtocolVersion); - } - first = false; - if (isSetSessionHandle()) { - if (!first) sb.append(", "); - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - } - if (isSetConfiguration()) { - if (!first) sb.append(", "); - sb.append("configuration:"); - if (this.configuration == null) { - sb.append("null"); - } else { - sb.append(this.configuration); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!isSetServerProtocolVersion()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverProtocolVersion' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOpenSessionRespStandardSchemeFactory implements SchemeFactory { - public TOpenSessionRespStandardScheme getScheme() { - return new TOpenSessionRespStandardScheme(); - } - } - - private static class TOpenSessionRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // SERVER_PROTOCOL_VERSION - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); - struct.setServerProtocolVersionIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // CONFIGURATION - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map152.size); - for (int _i153 = 0; _i153 < _map152.size; ++_i153) - { - String _key154; // required - String _val155; // required - _key154 = iprot.readString(); - _val155 = iprot.readString(); - struct.configuration.put(_key154, _val155); - } - iprot.readMapEnd(); - } - struct.setConfigurationIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.serverProtocolVersion != null) { - oprot.writeFieldBegin(SERVER_PROTOCOL_VERSION_FIELD_DESC); - oprot.writeI32(struct.serverProtocolVersion.getValue()); - oprot.writeFieldEnd(); - } - if (struct.sessionHandle != null) { - if (struct.isSetSessionHandle()) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - } - if (struct.configuration != null) { - if (struct.isSetConfiguration()) { - oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter156 : struct.configuration.entrySet()) - { - oprot.writeString(_iter156.getKey()); - oprot.writeString(_iter156.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOpenSessionRespTupleSchemeFactory implements SchemeFactory { - public TOpenSessionRespTupleScheme getScheme() { - return new TOpenSessionRespTupleScheme(); - } - } - - private static class TOpenSessionRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - oprot.writeI32(struct.serverProtocolVersion.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetSessionHandle()) { - optionals.set(0); - } - if (struct.isSetConfiguration()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetSessionHandle()) { - struct.sessionHandle.write(oprot); - } - if (struct.isSetConfiguration()) { - { - oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter157 : struct.configuration.entrySet()) - { - oprot.writeString(_iter157.getKey()); - oprot.writeString(_iter157.getValue()); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); - struct.setServerProtocolVersionIsSet(true); - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } - if (incoming.get(1)) { - { - org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map158.size); - for (int _i159 = 0; _i159 < _map158.size; ++_i159) - { - String _key160; // required - String _val161; // required - _key160 = iprot.readString(); - _val161 = iprot.readString(); - struct.configuration.put(_key160, _val161); - } - } - struct.setConfigurationIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java deleted file mode 100644 index 8fbd8752eaca6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java +++ /dev/null @@ -1,705 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TOperationHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOperationHandle"); - - private static final org.apache.thrift.protocol.TField OPERATION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("operationId", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)2); - private static final org.apache.thrift.protocol.TField HAS_RESULT_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("hasResultSet", org.apache.thrift.protocol.TType.BOOL, (short)3); - private static final org.apache.thrift.protocol.TField MODIFIED_ROW_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("modifiedRowCount", org.apache.thrift.protocol.TType.DOUBLE, (short)4); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TOperationHandleStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TOperationHandleTupleSchemeFactory()); - } - - private THandleIdentifier operationId; // required - private TOperationType operationType; // required - private boolean hasResultSet; // required - private double modifiedRowCount; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - OPERATION_ID((short)1, "operationId"), - /** - * - * @see TOperationType - */ - OPERATION_TYPE((short)2, "operationType"), - HAS_RESULT_SET((short)3, "hasResultSet"), - MODIFIED_ROW_COUNT((short)4, "modifiedRowCount"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OPERATION_ID - return OPERATION_ID; - case 2: // OPERATION_TYPE - return OPERATION_TYPE; - case 3: // HAS_RESULT_SET - return HAS_RESULT_SET; - case 4: // MODIFIED_ROW_COUNT - return MODIFIED_ROW_COUNT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __HASRESULTSET_ISSET_ID = 0; - private static final int __MODIFIEDROWCOUNT_ISSET_ID = 1; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.MODIFIED_ROW_COUNT}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OPERATION_ID, new org.apache.thrift.meta_data.FieldMetaData("operationId", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); - tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationType.class))); - tmpMap.put(_Fields.HAS_RESULT_SET, new org.apache.thrift.meta_data.FieldMetaData("hasResultSet", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); - tmpMap.put(_Fields.MODIFIED_ROW_COUNT, new org.apache.thrift.meta_data.FieldMetaData("modifiedRowCount", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOperationHandle.class, metaDataMap); - } - - public TOperationHandle() { - } - - public TOperationHandle( - THandleIdentifier operationId, - TOperationType operationType, - boolean hasResultSet) - { - this(); - this.operationId = operationId; - this.operationType = operationType; - this.hasResultSet = hasResultSet; - setHasResultSetIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TOperationHandle(TOperationHandle other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetOperationId()) { - this.operationId = new THandleIdentifier(other.operationId); - } - if (other.isSetOperationType()) { - this.operationType = other.operationType; - } - this.hasResultSet = other.hasResultSet; - this.modifiedRowCount = other.modifiedRowCount; - } - - public TOperationHandle deepCopy() { - return new TOperationHandle(this); - } - - @Override - public void clear() { - this.operationId = null; - this.operationType = null; - setHasResultSetIsSet(false); - this.hasResultSet = false; - setModifiedRowCountIsSet(false); - this.modifiedRowCount = 0.0; - } - - public THandleIdentifier getOperationId() { - return this.operationId; - } - - public void setOperationId(THandleIdentifier operationId) { - this.operationId = operationId; - } - - public void unsetOperationId() { - this.operationId = null; - } - - /** Returns true if field operationId is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationId() { - return this.operationId != null; - } - - public void setOperationIdIsSet(boolean value) { - if (!value) { - this.operationId = null; - } - } - - /** - * - * @see TOperationType - */ - public TOperationType getOperationType() { - return this.operationType; - } - - /** - * - * @see TOperationType - */ - public void setOperationType(TOperationType operationType) { - this.operationType = operationType; - } - - public void unsetOperationType() { - this.operationType = null; - } - - /** Returns true if field operationType is set (has been assigned a value) and false otherwise */ - public boolean isSetOperationType() { - return this.operationType != null; - } - - public void setOperationTypeIsSet(boolean value) { - if (!value) { - this.operationType = null; - } - } - - public boolean isHasResultSet() { - return this.hasResultSet; - } - - public void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - setHasResultSetIsSet(true); - } - - public void unsetHasResultSet() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); - } - - /** Returns true if field hasResultSet is set (has been assigned a value) and false otherwise */ - public boolean isSetHasResultSet() { - return EncodingUtils.testBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); - } - - public void setHasResultSetIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASRESULTSET_ISSET_ID, value); - } - - public double getModifiedRowCount() { - return this.modifiedRowCount; - } - - public void setModifiedRowCount(double modifiedRowCount) { - this.modifiedRowCount = modifiedRowCount; - setModifiedRowCountIsSet(true); - } - - public void unsetModifiedRowCount() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); - } - - /** Returns true if field modifiedRowCount is set (has been assigned a value) and false otherwise */ - public boolean isSetModifiedRowCount() { - return EncodingUtils.testBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); - } - - public void setModifiedRowCountIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OPERATION_ID: - if (value == null) { - unsetOperationId(); - } else { - setOperationId((THandleIdentifier)value); - } - break; - - case OPERATION_TYPE: - if (value == null) { - unsetOperationType(); - } else { - setOperationType((TOperationType)value); - } - break; - - case HAS_RESULT_SET: - if (value == null) { - unsetHasResultSet(); - } else { - setHasResultSet((Boolean)value); - } - break; - - case MODIFIED_ROW_COUNT: - if (value == null) { - unsetModifiedRowCount(); - } else { - setModifiedRowCount((Double)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OPERATION_ID: - return getOperationId(); - - case OPERATION_TYPE: - return getOperationType(); - - case HAS_RESULT_SET: - return Boolean.valueOf(isHasResultSet()); - - case MODIFIED_ROW_COUNT: - return Double.valueOf(getModifiedRowCount()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OPERATION_ID: - return isSetOperationId(); - case OPERATION_TYPE: - return isSetOperationType(); - case HAS_RESULT_SET: - return isSetHasResultSet(); - case MODIFIED_ROW_COUNT: - return isSetModifiedRowCount(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TOperationHandle) - return this.equals((TOperationHandle)that); - return false; - } - - public boolean equals(TOperationHandle that) { - if (that == null) - return false; - - boolean this_present_operationId = true && this.isSetOperationId(); - boolean that_present_operationId = true && that.isSetOperationId(); - if (this_present_operationId || that_present_operationId) { - if (!(this_present_operationId && that_present_operationId)) - return false; - if (!this.operationId.equals(that.operationId)) - return false; - } - - boolean this_present_operationType = true && this.isSetOperationType(); - boolean that_present_operationType = true && that.isSetOperationType(); - if (this_present_operationType || that_present_operationType) { - if (!(this_present_operationType && that_present_operationType)) - return false; - if (!this.operationType.equals(that.operationType)) - return false; - } - - boolean this_present_hasResultSet = true; - boolean that_present_hasResultSet = true; - if (this_present_hasResultSet || that_present_hasResultSet) { - if (!(this_present_hasResultSet && that_present_hasResultSet)) - return false; - if (this.hasResultSet != that.hasResultSet) - return false; - } - - boolean this_present_modifiedRowCount = true && this.isSetModifiedRowCount(); - boolean that_present_modifiedRowCount = true && that.isSetModifiedRowCount(); - if (this_present_modifiedRowCount || that_present_modifiedRowCount) { - if (!(this_present_modifiedRowCount && that_present_modifiedRowCount)) - return false; - if (this.modifiedRowCount != that.modifiedRowCount) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_operationId = true && (isSetOperationId()); - builder.append(present_operationId); - if (present_operationId) - builder.append(operationId); - - boolean present_operationType = true && (isSetOperationType()); - builder.append(present_operationType); - if (present_operationType) - builder.append(operationType.getValue()); - - boolean present_hasResultSet = true; - builder.append(present_hasResultSet); - if (present_hasResultSet) - builder.append(hasResultSet); - - boolean present_modifiedRowCount = true && (isSetModifiedRowCount()); - builder.append(present_modifiedRowCount); - if (present_modifiedRowCount) - builder.append(modifiedRowCount); - - return builder.toHashCode(); - } - - public int compareTo(TOperationHandle other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TOperationHandle typedOther = (TOperationHandle)other; - - lastComparison = Boolean.valueOf(isSetOperationId()).compareTo(typedOther.isSetOperationId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationId, typedOther.operationId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(typedOther.isSetOperationType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetOperationType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, typedOther.operationType); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetHasResultSet()).compareTo(typedOther.isSetHasResultSet()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetHasResultSet()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasResultSet, typedOther.hasResultSet); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetModifiedRowCount()).compareTo(typedOther.isSetModifiedRowCount()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetModifiedRowCount()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modifiedRowCount, typedOther.modifiedRowCount); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TOperationHandle("); - boolean first = true; - - sb.append("operationId:"); - if (this.operationId == null) { - sb.append("null"); - } else { - sb.append(this.operationId); - } - first = false; - if (!first) sb.append(", "); - sb.append("operationType:"); - if (this.operationType == null) { - sb.append("null"); - } else { - sb.append(this.operationType); - } - first = false; - if (!first) sb.append(", "); - sb.append("hasResultSet:"); - sb.append(this.hasResultSet); - first = false; - if (isSetModifiedRowCount()) { - if (!first) sb.append(", "); - sb.append("modifiedRowCount:"); - sb.append(this.modifiedRowCount); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetOperationId()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationId' is unset! Struct:" + toString()); - } - - if (!isSetOperationType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationType' is unset! Struct:" + toString()); - } - - if (!isSetHasResultSet()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'hasResultSet' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (operationId != null) { - operationId.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TOperationHandleStandardSchemeFactory implements SchemeFactory { - public TOperationHandleStandardScheme getScheme() { - return new TOperationHandleStandardScheme(); - } - } - - private static class TOperationHandleStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TOperationHandle struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // OPERATION_ID - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.operationId = new THandleIdentifier(); - struct.operationId.read(iprot); - struct.setOperationIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // OPERATION_TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.operationType = TOperationType.findByValue(iprot.readI32()); - struct.setOperationTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // HAS_RESULT_SET - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.hasResultSet = iprot.readBool(); - struct.setHasResultSetIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // MODIFIED_ROW_COUNT - if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { - struct.modifiedRowCount = iprot.readDouble(); - struct.setModifiedRowCountIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TOperationHandle struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.operationId != null) { - oprot.writeFieldBegin(OPERATION_ID_FIELD_DESC); - struct.operationId.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.operationType != null) { - oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC); - oprot.writeI32(struct.operationType.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(HAS_RESULT_SET_FIELD_DESC); - oprot.writeBool(struct.hasResultSet); - oprot.writeFieldEnd(); - if (struct.isSetModifiedRowCount()) { - oprot.writeFieldBegin(MODIFIED_ROW_COUNT_FIELD_DESC); - oprot.writeDouble(struct.modifiedRowCount); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TOperationHandleTupleSchemeFactory implements SchemeFactory { - public TOperationHandleTupleScheme getScheme() { - return new TOperationHandleTupleScheme(); - } - } - - private static class TOperationHandleTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.operationId.write(oprot); - oprot.writeI32(struct.operationType.getValue()); - oprot.writeBool(struct.hasResultSet); - BitSet optionals = new BitSet(); - if (struct.isSetModifiedRowCount()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetModifiedRowCount()) { - oprot.writeDouble(struct.modifiedRowCount); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.operationId = new THandleIdentifier(); - struct.operationId.read(iprot); - struct.setOperationIdIsSet(true); - struct.operationType = TOperationType.findByValue(iprot.readI32()); - struct.setOperationTypeIsSet(true); - struct.hasResultSet = iprot.readBool(); - struct.setHasResultSetIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.modifiedRowCount = iprot.readDouble(); - struct.setModifiedRowCountIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java deleted file mode 100644 index 219866223a6b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TOperationState implements org.apache.thrift.TEnum { - INITIALIZED_STATE(0), - RUNNING_STATE(1), - FINISHED_STATE(2), - CANCELED_STATE(3), - CLOSED_STATE(4), - ERROR_STATE(5), - UKNOWN_STATE(6), - PENDING_STATE(7); - - private final int value; - - private TOperationState(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TOperationState findByValue(int value) { - switch (value) { - case 0: - return INITIALIZED_STATE; - case 1: - return RUNNING_STATE; - case 2: - return FINISHED_STATE; - case 3: - return CANCELED_STATE; - case 4: - return CLOSED_STATE; - case 5: - return ERROR_STATE; - case 6: - return UKNOWN_STATE; - case 7: - return PENDING_STATE; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java deleted file mode 100644 index b6d4b2fab9f96..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TOperationType implements org.apache.thrift.TEnum { - EXECUTE_STATEMENT(0), - GET_TYPE_INFO(1), - GET_CATALOGS(2), - GET_SCHEMAS(3), - GET_TABLES(4), - GET_TABLE_TYPES(5), - GET_COLUMNS(6), - GET_FUNCTIONS(7), - UNKNOWN(8); - - private final int value; - - private TOperationType(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TOperationType findByValue(int value) { - switch (value) { - case 0: - return EXECUTE_STATEMENT; - case 1: - return GET_TYPE_INFO; - case 2: - return GET_CATALOGS; - case 3: - return GET_SCHEMAS; - case 4: - return GET_TABLES; - case 5: - return GET_TABLE_TYPES; - case 6: - return GET_COLUMNS; - case 7: - return GET_FUNCTIONS; - case 8: - return UNKNOWN; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java deleted file mode 100644 index 9d2abf2b3b084..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java +++ /dev/null @@ -1,512 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TPrimitiveTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPrimitiveTypeEntry"); - - private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField TYPE_QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("typeQualifiers", org.apache.thrift.protocol.TType.STRUCT, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TPrimitiveTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TPrimitiveTypeEntryTupleSchemeFactory()); - } - - private TTypeId type; // required - private TTypeQualifiers typeQualifiers; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TTypeId - */ - TYPE((short)1, "type"), - TYPE_QUALIFIERS((short)2, "typeQualifiers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPE - return TYPE; - case 2: // TYPE_QUALIFIERS - return TYPE_QUALIFIERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.TYPE_QUALIFIERS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTypeId.class))); - tmpMap.put(_Fields.TYPE_QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("typeQualifiers", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifiers.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TPrimitiveTypeEntry.class, metaDataMap); - } - - public TPrimitiveTypeEntry() { - } - - public TPrimitiveTypeEntry( - TTypeId type) - { - this(); - this.type = type; - } - - /** - * Performs a deep copy on other. - */ - public TPrimitiveTypeEntry(TPrimitiveTypeEntry other) { - if (other.isSetType()) { - this.type = other.type; - } - if (other.isSetTypeQualifiers()) { - this.typeQualifiers = new TTypeQualifiers(other.typeQualifiers); - } - } - - public TPrimitiveTypeEntry deepCopy() { - return new TPrimitiveTypeEntry(this); - } - - @Override - public void clear() { - this.type = null; - this.typeQualifiers = null; - } - - /** - * - * @see TTypeId - */ - public TTypeId getType() { - return this.type; - } - - /** - * - * @see TTypeId - */ - public void setType(TTypeId type) { - this.type = type; - } - - public void unsetType() { - this.type = null; - } - - /** Returns true if field type is set (has been assigned a value) and false otherwise */ - public boolean isSetType() { - return this.type != null; - } - - public void setTypeIsSet(boolean value) { - if (!value) { - this.type = null; - } - } - - public TTypeQualifiers getTypeQualifiers() { - return this.typeQualifiers; - } - - public void setTypeQualifiers(TTypeQualifiers typeQualifiers) { - this.typeQualifiers = typeQualifiers; - } - - public void unsetTypeQualifiers() { - this.typeQualifiers = null; - } - - /** Returns true if field typeQualifiers is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeQualifiers() { - return this.typeQualifiers != null; - } - - public void setTypeQualifiersIsSet(boolean value) { - if (!value) { - this.typeQualifiers = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPE: - if (value == null) { - unsetType(); - } else { - setType((TTypeId)value); - } - break; - - case TYPE_QUALIFIERS: - if (value == null) { - unsetTypeQualifiers(); - } else { - setTypeQualifiers((TTypeQualifiers)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPE: - return getType(); - - case TYPE_QUALIFIERS: - return getTypeQualifiers(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPE: - return isSetType(); - case TYPE_QUALIFIERS: - return isSetTypeQualifiers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TPrimitiveTypeEntry) - return this.equals((TPrimitiveTypeEntry)that); - return false; - } - - public boolean equals(TPrimitiveTypeEntry that) { - if (that == null) - return false; - - boolean this_present_type = true && this.isSetType(); - boolean that_present_type = true && that.isSetType(); - if (this_present_type || that_present_type) { - if (!(this_present_type && that_present_type)) - return false; - if (!this.type.equals(that.type)) - return false; - } - - boolean this_present_typeQualifiers = true && this.isSetTypeQualifiers(); - boolean that_present_typeQualifiers = true && that.isSetTypeQualifiers(); - if (this_present_typeQualifiers || that_present_typeQualifiers) { - if (!(this_present_typeQualifiers && that_present_typeQualifiers)) - return false; - if (!this.typeQualifiers.equals(that.typeQualifiers)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_type = true && (isSetType()); - builder.append(present_type); - if (present_type) - builder.append(type.getValue()); - - boolean present_typeQualifiers = true && (isSetTypeQualifiers()); - builder.append(present_typeQualifiers); - if (present_typeQualifiers) - builder.append(typeQualifiers); - - return builder.toHashCode(); - } - - public int compareTo(TPrimitiveTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TPrimitiveTypeEntry typedOther = (TPrimitiveTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetType()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetTypeQualifiers()).compareTo(typedOther.isSetTypeQualifiers()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeQualifiers()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeQualifiers, typedOther.typeQualifiers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TPrimitiveTypeEntry("); - boolean first = true; - - sb.append("type:"); - if (this.type == null) { - sb.append("null"); - } else { - sb.append(this.type); - } - first = false; - if (isSetTypeQualifiers()) { - if (!first) sb.append(", "); - sb.append("typeQualifiers:"); - if (this.typeQualifiers == null) { - sb.append("null"); - } else { - sb.append(this.typeQualifiers); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetType()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (typeQualifiers != null) { - typeQualifiers.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TPrimitiveTypeEntryStandardSchemeFactory implements SchemeFactory { - public TPrimitiveTypeEntryStandardScheme getScheme() { - return new TPrimitiveTypeEntryStandardScheme(); - } - } - - private static class TPrimitiveTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.type = TTypeId.findByValue(iprot.readI32()); - struct.setTypeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // TYPE_QUALIFIERS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.typeQualifiers = new TTypeQualifiers(); - struct.typeQualifiers.read(iprot); - struct.setTypeQualifiersIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.type != null) { - oprot.writeFieldBegin(TYPE_FIELD_DESC); - oprot.writeI32(struct.type.getValue()); - oprot.writeFieldEnd(); - } - if (struct.typeQualifiers != null) { - if (struct.isSetTypeQualifiers()) { - oprot.writeFieldBegin(TYPE_QUALIFIERS_FIELD_DESC); - struct.typeQualifiers.write(oprot); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TPrimitiveTypeEntryTupleSchemeFactory implements SchemeFactory { - public TPrimitiveTypeEntryTupleScheme getScheme() { - return new TPrimitiveTypeEntryTupleScheme(); - } - } - - private static class TPrimitiveTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.type.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetTypeQualifiers()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetTypeQualifiers()) { - struct.typeQualifiers.write(oprot); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.type = TTypeId.findByValue(iprot.readI32()); - struct.setTypeIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.typeQualifiers = new TTypeQualifiers(); - struct.typeQualifiers.read(iprot); - struct.setTypeQualifiersIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java deleted file mode 100644 index a4279d29f662e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TProtocolVersion implements org.apache.thrift.TEnum { - HIVE_CLI_SERVICE_PROTOCOL_V1(0), - HIVE_CLI_SERVICE_PROTOCOL_V2(1), - HIVE_CLI_SERVICE_PROTOCOL_V3(2), - HIVE_CLI_SERVICE_PROTOCOL_V4(3), - HIVE_CLI_SERVICE_PROTOCOL_V5(4), - HIVE_CLI_SERVICE_PROTOCOL_V6(5), - HIVE_CLI_SERVICE_PROTOCOL_V7(6), - HIVE_CLI_SERVICE_PROTOCOL_V8(7); - - private final int value; - - private TProtocolVersion(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TProtocolVersion findByValue(int value) { - switch (value) { - case 0: - return HIVE_CLI_SERVICE_PROTOCOL_V1; - case 1: - return HIVE_CLI_SERVICE_PROTOCOL_V2; - case 2: - return HIVE_CLI_SERVICE_PROTOCOL_V3; - case 3: - return HIVE_CLI_SERVICE_PROTOCOL_V4; - case 4: - return HIVE_CLI_SERVICE_PROTOCOL_V5; - case 5: - return HIVE_CLI_SERVICE_PROTOCOL_V6; - case 6: - return HIVE_CLI_SERVICE_PROTOCOL_V7; - case 7: - return HIVE_CLI_SERVICE_PROTOCOL_V8; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java deleted file mode 100644 index a3e39c8cdf321..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java +++ /dev/null @@ -1,491 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRenewDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenReq"); - - private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRenewDelegationTokenReqStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRenewDelegationTokenReqTupleSchemeFactory()); - } - - private TSessionHandle sessionHandle; // required - private String delegationToken; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_HANDLE((short)1, "sessionHandle"), - DELEGATION_TOKEN((short)2, "delegationToken"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_HANDLE - return SESSION_HANDLE; - case 2: // DELEGATION_TOKEN - return DELEGATION_TOKEN; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); - tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenReq.class, metaDataMap); - } - - public TRenewDelegationTokenReq() { - } - - public TRenewDelegationTokenReq( - TSessionHandle sessionHandle, - String delegationToken) - { - this(); - this.sessionHandle = sessionHandle; - this.delegationToken = delegationToken; - } - - /** - * Performs a deep copy on other. - */ - public TRenewDelegationTokenReq(TRenewDelegationTokenReq other) { - if (other.isSetSessionHandle()) { - this.sessionHandle = new TSessionHandle(other.sessionHandle); - } - if (other.isSetDelegationToken()) { - this.delegationToken = other.delegationToken; - } - } - - public TRenewDelegationTokenReq deepCopy() { - return new TRenewDelegationTokenReq(this); - } - - @Override - public void clear() { - this.sessionHandle = null; - this.delegationToken = null; - } - - public TSessionHandle getSessionHandle() { - return this.sessionHandle; - } - - public void setSessionHandle(TSessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public void unsetSessionHandle() { - this.sessionHandle = null; - } - - /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionHandle() { - return this.sessionHandle != null; - } - - public void setSessionHandleIsSet(boolean value) { - if (!value) { - this.sessionHandle = null; - } - } - - public String getDelegationToken() { - return this.delegationToken; - } - - public void setDelegationToken(String delegationToken) { - this.delegationToken = delegationToken; - } - - public void unsetDelegationToken() { - this.delegationToken = null; - } - - /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ - public boolean isSetDelegationToken() { - return this.delegationToken != null; - } - - public void setDelegationTokenIsSet(boolean value) { - if (!value) { - this.delegationToken = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_HANDLE: - if (value == null) { - unsetSessionHandle(); - } else { - setSessionHandle((TSessionHandle)value); - } - break; - - case DELEGATION_TOKEN: - if (value == null) { - unsetDelegationToken(); - } else { - setDelegationToken((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_HANDLE: - return getSessionHandle(); - - case DELEGATION_TOKEN: - return getDelegationToken(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_HANDLE: - return isSetSessionHandle(); - case DELEGATION_TOKEN: - return isSetDelegationToken(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRenewDelegationTokenReq) - return this.equals((TRenewDelegationTokenReq)that); - return false; - } - - public boolean equals(TRenewDelegationTokenReq that) { - if (that == null) - return false; - - boolean this_present_sessionHandle = true && this.isSetSessionHandle(); - boolean that_present_sessionHandle = true && that.isSetSessionHandle(); - if (this_present_sessionHandle || that_present_sessionHandle) { - if (!(this_present_sessionHandle && that_present_sessionHandle)) - return false; - if (!this.sessionHandle.equals(that.sessionHandle)) - return false; - } - - boolean this_present_delegationToken = true && this.isSetDelegationToken(); - boolean that_present_delegationToken = true && that.isSetDelegationToken(); - if (this_present_delegationToken || that_present_delegationToken) { - if (!(this_present_delegationToken && that_present_delegationToken)) - return false; - if (!this.delegationToken.equals(that.delegationToken)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionHandle = true && (isSetSessionHandle()); - builder.append(present_sessionHandle); - if (present_sessionHandle) - builder.append(sessionHandle); - - boolean present_delegationToken = true && (isSetDelegationToken()); - builder.append(present_delegationToken); - if (present_delegationToken) - builder.append(delegationToken); - - return builder.toHashCode(); - } - - public int compareTo(TRenewDelegationTokenReq other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRenewDelegationTokenReq typedOther = (TRenewDelegationTokenReq)other; - - lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionHandle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetDelegationToken()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRenewDelegationTokenReq("); - boolean first = true; - - sb.append("sessionHandle:"); - if (this.sessionHandle == null) { - sb.append("null"); - } else { - sb.append(this.sessionHandle); - } - first = false; - if (!first) sb.append(", "); - sb.append("delegationToken:"); - if (this.delegationToken == null) { - sb.append("null"); - } else { - sb.append(this.delegationToken); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionHandle()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); - } - - if (!isSetDelegationToken()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionHandle != null) { - sessionHandle.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRenewDelegationTokenReqStandardSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenReqStandardScheme getScheme() { - return new TRenewDelegationTokenReqStandardScheme(); - } - } - - private static class TRenewDelegationTokenReqStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_HANDLE - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // DELEGATION_TOKEN - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionHandle != null) { - oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); - struct.sessionHandle.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.delegationToken != null) { - oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); - oprot.writeString(struct.delegationToken); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRenewDelegationTokenReqTupleSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenReqTupleScheme getScheme() { - return new TRenewDelegationTokenReqTupleScheme(); - } - } - - private static class TRenewDelegationTokenReqTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionHandle.write(oprot); - oprot.writeString(struct.delegationToken); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionHandle = new TSessionHandle(); - struct.sessionHandle.read(iprot); - struct.setSessionHandleIsSet(true); - struct.delegationToken = iprot.readString(); - struct.setDelegationTokenIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java deleted file mode 100644 index 5f3eb6c4d4b90..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRenewDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenResp"); - - private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRenewDelegationTokenRespStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRenewDelegationTokenRespTupleSchemeFactory()); - } - - private TStatus status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STATUS((short)1, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenResp.class, metaDataMap); - } - - public TRenewDelegationTokenResp() { - } - - public TRenewDelegationTokenResp( - TStatus status) - { - this(); - this.status = status; - } - - /** - * Performs a deep copy on other. - */ - public TRenewDelegationTokenResp(TRenewDelegationTokenResp other) { - if (other.isSetStatus()) { - this.status = new TStatus(other.status); - } - } - - public TRenewDelegationTokenResp deepCopy() { - return new TRenewDelegationTokenResp(this); - } - - @Override - public void clear() { - this.status = null; - } - - public TStatus getStatus() { - return this.status; - } - - public void setStatus(TStatus status) { - this.status = status; - } - - public void unsetStatus() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean isSetStatus() { - return this.status != null; - } - - public void setStatusIsSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS: - if (value == null) { - unsetStatus(); - } else { - setStatus((TStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS: - return getStatus(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS: - return isSetStatus(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRenewDelegationTokenResp) - return this.equals((TRenewDelegationTokenResp)that); - return false; - } - - public boolean equals(TRenewDelegationTokenResp that) { - if (that == null) - return false; - - boolean this_present_status = true && this.isSetStatus(); - boolean that_present_status = true && that.isSetStatus(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_status = true && (isSetStatus()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TRenewDelegationTokenResp other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRenewDelegationTokenResp typedOther = (TRenewDelegationTokenResp)other; - - lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatus()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRenewDelegationTokenResp("); - boolean first = true; - - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatus()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (status != null) { - status.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRenewDelegationTokenRespStandardSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenRespStandardScheme getScheme() { - return new TRenewDelegationTokenRespStandardScheme(); - } - } - - private static class TRenewDelegationTokenRespStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - struct.status.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRenewDelegationTokenRespTupleSchemeFactory implements SchemeFactory { - public TRenewDelegationTokenRespTupleScheme getScheme() { - return new TRenewDelegationTokenRespTupleScheme(); - } - } - - private static class TRenewDelegationTokenRespTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.status.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.status = new TStatus(); - struct.status.read(iprot); - struct.setStatusIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java deleted file mode 100644 index a44cfb08ff01a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRow implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRow"); - - private static final org.apache.thrift.protocol.TField COL_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("colVals", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRowStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRowTupleSchemeFactory()); - } - - private List colVals; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COL_VALS((short)1, "colVals"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COL_VALS - return COL_VALS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COL_VALS, new org.apache.thrift.meta_data.FieldMetaData("colVals", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnValue.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRow.class, metaDataMap); - } - - public TRow() { - } - - public TRow( - List colVals) - { - this(); - this.colVals = colVals; - } - - /** - * Performs a deep copy on other. - */ - public TRow(TRow other) { - if (other.isSetColVals()) { - List __this__colVals = new ArrayList(); - for (TColumnValue other_element : other.colVals) { - __this__colVals.add(new TColumnValue(other_element)); - } - this.colVals = __this__colVals; - } - } - - public TRow deepCopy() { - return new TRow(this); - } - - @Override - public void clear() { - this.colVals = null; - } - - public int getColValsSize() { - return (this.colVals == null) ? 0 : this.colVals.size(); - } - - public java.util.Iterator getColValsIterator() { - return (this.colVals == null) ? null : this.colVals.iterator(); - } - - public void addToColVals(TColumnValue elem) { - if (this.colVals == null) { - this.colVals = new ArrayList(); - } - this.colVals.add(elem); - } - - public List getColVals() { - return this.colVals; - } - - public void setColVals(List colVals) { - this.colVals = colVals; - } - - public void unsetColVals() { - this.colVals = null; - } - - /** Returns true if field colVals is set (has been assigned a value) and false otherwise */ - public boolean isSetColVals() { - return this.colVals != null; - } - - public void setColValsIsSet(boolean value) { - if (!value) { - this.colVals = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COL_VALS: - if (value == null) { - unsetColVals(); - } else { - setColVals((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COL_VALS: - return getColVals(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COL_VALS: - return isSetColVals(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRow) - return this.equals((TRow)that); - return false; - } - - public boolean equals(TRow that) { - if (that == null) - return false; - - boolean this_present_colVals = true && this.isSetColVals(); - boolean that_present_colVals = true && that.isSetColVals(); - if (this_present_colVals || that_present_colVals) { - if (!(this_present_colVals && that_present_colVals)) - return false; - if (!this.colVals.equals(that.colVals)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_colVals = true && (isSetColVals()); - builder.append(present_colVals); - if (present_colVals) - builder.append(colVals); - - return builder.toHashCode(); - } - - public int compareTo(TRow other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRow typedOther = (TRow)other; - - lastComparison = Boolean.valueOf(isSetColVals()).compareTo(typedOther.isSetColVals()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColVals()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colVals, typedOther.colVals); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRow("); - boolean first = true; - - sb.append("colVals:"); - if (this.colVals == null) { - sb.append("null"); - } else { - sb.append(this.colVals); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColVals()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'colVals' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRowStandardSchemeFactory implements SchemeFactory { - public TRowStandardScheme getScheme() { - return new TRowStandardScheme(); - } - } - - private static class TRowStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRow struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COL_VALS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list46 = iprot.readListBegin(); - struct.colVals = new ArrayList(_list46.size); - for (int _i47 = 0; _i47 < _list46.size; ++_i47) - { - TColumnValue _elem48; // optional - _elem48 = new TColumnValue(); - _elem48.read(iprot); - struct.colVals.add(_elem48); - } - iprot.readListEnd(); - } - struct.setColValsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRow struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.colVals != null) { - oprot.writeFieldBegin(COL_VALS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colVals.size())); - for (TColumnValue _iter49 : struct.colVals) - { - _iter49.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRowTupleSchemeFactory implements SchemeFactory { - public TRowTupleScheme getScheme() { - return new TRowTupleScheme(); - } - } - - private static class TRowTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.colVals.size()); - for (TColumnValue _iter50 : struct.colVals) - { - _iter50.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list51 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.colVals = new ArrayList(_list51.size); - for (int _i52 = 0; _i52 < _list51.size; ++_i52) - { - TColumnValue _elem53; // optional - _elem53 = new TColumnValue(); - _elem53.read(iprot); - struct.colVals.add(_elem53); - } - } - struct.setColValsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java deleted file mode 100644 index d16c8a4bb32da..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java +++ /dev/null @@ -1,702 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TRowSet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowSet"); - - private static final org.apache.thrift.protocol.TField START_ROW_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("startRowOffset", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); - private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TRowSetStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TRowSetTupleSchemeFactory()); - } - - private long startRowOffset; // required - private List rows; // required - private List columns; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - START_ROW_OFFSET((short)1, "startRowOffset"), - ROWS((short)2, "rows"), - COLUMNS((short)3, "columns"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // START_ROW_OFFSET - return START_ROW_OFFSET; - case 2: // ROWS - return ROWS; - case 3: // COLUMNS - return COLUMNS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __STARTROWOFFSET_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.COLUMNS}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.START_ROW_OFFSET, new org.apache.thrift.meta_data.FieldMetaData("startRowOffset", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRow.class)))); - tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumn.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowSet.class, metaDataMap); - } - - public TRowSet() { - } - - public TRowSet( - long startRowOffset, - List rows) - { - this(); - this.startRowOffset = startRowOffset; - setStartRowOffsetIsSet(true); - this.rows = rows; - } - - /** - * Performs a deep copy on other. - */ - public TRowSet(TRowSet other) { - __isset_bitfield = other.__isset_bitfield; - this.startRowOffset = other.startRowOffset; - if (other.isSetRows()) { - List __this__rows = new ArrayList(); - for (TRow other_element : other.rows) { - __this__rows.add(new TRow(other_element)); - } - this.rows = __this__rows; - } - if (other.isSetColumns()) { - List __this__columns = new ArrayList(); - for (TColumn other_element : other.columns) { - __this__columns.add(new TColumn(other_element)); - } - this.columns = __this__columns; - } - } - - public TRowSet deepCopy() { - return new TRowSet(this); - } - - @Override - public void clear() { - setStartRowOffsetIsSet(false); - this.startRowOffset = 0; - this.rows = null; - this.columns = null; - } - - public long getStartRowOffset() { - return this.startRowOffset; - } - - public void setStartRowOffset(long startRowOffset) { - this.startRowOffset = startRowOffset; - setStartRowOffsetIsSet(true); - } - - public void unsetStartRowOffset() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); - } - - /** Returns true if field startRowOffset is set (has been assigned a value) and false otherwise */ - public boolean isSetStartRowOffset() { - return EncodingUtils.testBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); - } - - public void setStartRowOffsetIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID, value); - } - - public int getRowsSize() { - return (this.rows == null) ? 0 : this.rows.size(); - } - - public java.util.Iterator getRowsIterator() { - return (this.rows == null) ? null : this.rows.iterator(); - } - - public void addToRows(TRow elem) { - if (this.rows == null) { - this.rows = new ArrayList(); - } - this.rows.add(elem); - } - - public List getRows() { - return this.rows; - } - - public void setRows(List rows) { - this.rows = rows; - } - - public void unsetRows() { - this.rows = null; - } - - /** Returns true if field rows is set (has been assigned a value) and false otherwise */ - public boolean isSetRows() { - return this.rows != null; - } - - public void setRowsIsSet(boolean value) { - if (!value) { - this.rows = null; - } - } - - public int getColumnsSize() { - return (this.columns == null) ? 0 : this.columns.size(); - } - - public java.util.Iterator getColumnsIterator() { - return (this.columns == null) ? null : this.columns.iterator(); - } - - public void addToColumns(TColumn elem) { - if (this.columns == null) { - this.columns = new ArrayList(); - } - this.columns.add(elem); - } - - public List getColumns() { - return this.columns; - } - - public void setColumns(List columns) { - this.columns = columns; - } - - public void unsetColumns() { - this.columns = null; - } - - /** Returns true if field columns is set (has been assigned a value) and false otherwise */ - public boolean isSetColumns() { - return this.columns != null; - } - - public void setColumnsIsSet(boolean value) { - if (!value) { - this.columns = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case START_ROW_OFFSET: - if (value == null) { - unsetStartRowOffset(); - } else { - setStartRowOffset((Long)value); - } - break; - - case ROWS: - if (value == null) { - unsetRows(); - } else { - setRows((List)value); - } - break; - - case COLUMNS: - if (value == null) { - unsetColumns(); - } else { - setColumns((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case START_ROW_OFFSET: - return Long.valueOf(getStartRowOffset()); - - case ROWS: - return getRows(); - - case COLUMNS: - return getColumns(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case START_ROW_OFFSET: - return isSetStartRowOffset(); - case ROWS: - return isSetRows(); - case COLUMNS: - return isSetColumns(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TRowSet) - return this.equals((TRowSet)that); - return false; - } - - public boolean equals(TRowSet that) { - if (that == null) - return false; - - boolean this_present_startRowOffset = true; - boolean that_present_startRowOffset = true; - if (this_present_startRowOffset || that_present_startRowOffset) { - if (!(this_present_startRowOffset && that_present_startRowOffset)) - return false; - if (this.startRowOffset != that.startRowOffset) - return false; - } - - boolean this_present_rows = true && this.isSetRows(); - boolean that_present_rows = true && that.isSetRows(); - if (this_present_rows || that_present_rows) { - if (!(this_present_rows && that_present_rows)) - return false; - if (!this.rows.equals(that.rows)) - return false; - } - - boolean this_present_columns = true && this.isSetColumns(); - boolean that_present_columns = true && that.isSetColumns(); - if (this_present_columns || that_present_columns) { - if (!(this_present_columns && that_present_columns)) - return false; - if (!this.columns.equals(that.columns)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_startRowOffset = true; - builder.append(present_startRowOffset); - if (present_startRowOffset) - builder.append(startRowOffset); - - boolean present_rows = true && (isSetRows()); - builder.append(present_rows); - if (present_rows) - builder.append(rows); - - boolean present_columns = true && (isSetColumns()); - builder.append(present_columns); - if (present_columns) - builder.append(columns); - - return builder.toHashCode(); - } - - public int compareTo(TRowSet other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TRowSet typedOther = (TRowSet)other; - - lastComparison = Boolean.valueOf(isSetStartRowOffset()).compareTo(typedOther.isSetStartRowOffset()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStartRowOffset()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRowOffset, typedOther.startRowOffset); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TRowSet("); - boolean first = true; - - sb.append("startRowOffset:"); - sb.append(this.startRowOffset); - first = false; - if (!first) sb.append(", "); - sb.append("rows:"); - if (this.rows == null) { - sb.append("null"); - } else { - sb.append(this.rows); - } - first = false; - if (isSetColumns()) { - if (!first) sb.append(", "); - sb.append("columns:"); - if (this.columns == null) { - sb.append("null"); - } else { - sb.append(this.columns); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStartRowOffset()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'startRowOffset' is unset! Struct:" + toString()); - } - - if (!isSetRows()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TRowSetStandardSchemeFactory implements SchemeFactory { - public TRowSetStandardScheme getScheme() { - return new TRowSetStandardScheme(); - } - } - - private static class TRowSetStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // START_ROW_OFFSET - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.startRowOffset = iprot.readI64(); - struct.setStartRowOffsetIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // ROWS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list118 = iprot.readListBegin(); - struct.rows = new ArrayList(_list118.size); - for (int _i119 = 0; _i119 < _list118.size; ++_i119) - { - TRow _elem120; // optional - _elem120 = new TRow(); - _elem120.read(iprot); - struct.rows.add(_elem120); - } - iprot.readListEnd(); - } - struct.setRowsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // COLUMNS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list121 = iprot.readListBegin(); - struct.columns = new ArrayList(_list121.size); - for (int _i122 = 0; _i122 < _list121.size; ++_i122) - { - TColumn _elem123; // optional - _elem123 = new TColumn(); - _elem123.read(iprot); - struct.columns.add(_elem123); - } - iprot.readListEnd(); - } - struct.setColumnsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(START_ROW_OFFSET_FIELD_DESC); - oprot.writeI64(struct.startRowOffset); - oprot.writeFieldEnd(); - if (struct.rows != null) { - oprot.writeFieldBegin(ROWS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rows.size())); - for (TRow _iter124 : struct.rows) - { - _iter124.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.columns != null) { - if (struct.isSetColumns()) { - oprot.writeFieldBegin(COLUMNS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); - for (TColumn _iter125 : struct.columns) - { - _iter125.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TRowSetTupleSchemeFactory implements SchemeFactory { - public TRowSetTupleScheme getScheme() { - return new TRowSetTupleScheme(); - } - } - - private static class TRowSetTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI64(struct.startRowOffset); - { - oprot.writeI32(struct.rows.size()); - for (TRow _iter126 : struct.rows) - { - _iter126.write(oprot); - } - } - BitSet optionals = new BitSet(); - if (struct.isSetColumns()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetColumns()) { - { - oprot.writeI32(struct.columns.size()); - for (TColumn _iter127 : struct.columns) - { - _iter127.write(oprot); - } - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.startRowOffset = iprot.readI64(); - struct.setStartRowOffsetIsSet(true); - { - org.apache.thrift.protocol.TList _list128 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.rows = new ArrayList(_list128.size); - for (int _i129 = 0; _i129 < _list128.size; ++_i129) - { - TRow _elem130; // optional - _elem130 = new TRow(); - _elem130.read(iprot); - struct.rows.add(_elem130); - } - } - struct.setRowsIsSet(true); - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columns = new ArrayList(_list131.size); - for (int _i132 = 0; _i132 < _list131.size; ++_i132) - { - TColumn _elem133; // optional - _elem133 = new TColumn(); - _elem133.read(iprot); - struct.columns.add(_elem133); - } - } - struct.setColumnsIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java deleted file mode 100644 index 82c00dd68a98b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java +++ /dev/null @@ -1,390 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TSessionHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSessionHandle"); - - private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TSessionHandleStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TSessionHandleTupleSchemeFactory()); - } - - private THandleIdentifier sessionId; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SESSION_ID((short)1, "sessionId"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SESSION_ID - return SESSION_ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSessionHandle.class, metaDataMap); - } - - public TSessionHandle() { - } - - public TSessionHandle( - THandleIdentifier sessionId) - { - this(); - this.sessionId = sessionId; - } - - /** - * Performs a deep copy on other. - */ - public TSessionHandle(TSessionHandle other) { - if (other.isSetSessionId()) { - this.sessionId = new THandleIdentifier(other.sessionId); - } - } - - public TSessionHandle deepCopy() { - return new TSessionHandle(this); - } - - @Override - public void clear() { - this.sessionId = null; - } - - public THandleIdentifier getSessionId() { - return this.sessionId; - } - - public void setSessionId(THandleIdentifier sessionId) { - this.sessionId = sessionId; - } - - public void unsetSessionId() { - this.sessionId = null; - } - - /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */ - public boolean isSetSessionId() { - return this.sessionId != null; - } - - public void setSessionIdIsSet(boolean value) { - if (!value) { - this.sessionId = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SESSION_ID: - if (value == null) { - unsetSessionId(); - } else { - setSessionId((THandleIdentifier)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SESSION_ID: - return getSessionId(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SESSION_ID: - return isSetSessionId(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TSessionHandle) - return this.equals((TSessionHandle)that); - return false; - } - - public boolean equals(TSessionHandle that) { - if (that == null) - return false; - - boolean this_present_sessionId = true && this.isSetSessionId(); - boolean that_present_sessionId = true && that.isSetSessionId(); - if (this_present_sessionId || that_present_sessionId) { - if (!(this_present_sessionId && that_present_sessionId)) - return false; - if (!this.sessionId.equals(that.sessionId)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_sessionId = true && (isSetSessionId()); - builder.append(present_sessionId); - if (present_sessionId) - builder.append(sessionId); - - return builder.toHashCode(); - } - - public int compareTo(TSessionHandle other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TSessionHandle typedOther = (TSessionHandle)other; - - lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(typedOther.isSetSessionId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSessionId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, typedOther.sessionId); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TSessionHandle("); - boolean first = true; - - sb.append("sessionId:"); - if (this.sessionId == null) { - sb.append("null"); - } else { - sb.append(this.sessionId); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetSessionId()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionId' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - if (sessionId != null) { - sessionId.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TSessionHandleStandardSchemeFactory implements SchemeFactory { - public TSessionHandleStandardScheme getScheme() { - return new TSessionHandleStandardScheme(); - } - } - - private static class TSessionHandleStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TSessionHandle struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // SESSION_ID - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.sessionId = new THandleIdentifier(); - struct.sessionId.read(iprot); - struct.setSessionIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TSessionHandle struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.sessionId != null) { - oprot.writeFieldBegin(SESSION_ID_FIELD_DESC); - struct.sessionId.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TSessionHandleTupleSchemeFactory implements SchemeFactory { - public TSessionHandleTupleScheme getScheme() { - return new TSessionHandleTupleScheme(); - } - } - - private static class TSessionHandleTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - struct.sessionId.write(oprot); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.sessionId = new THandleIdentifier(); - struct.sessionId.read(iprot); - struct.setSessionIdIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java deleted file mode 100644 index 24a746e94965d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java +++ /dev/null @@ -1,874 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStatus implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStatus"); - - private static final org.apache.thrift.protocol.TField STATUS_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("statusCode", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField INFO_MESSAGES_FIELD_DESC = new org.apache.thrift.protocol.TField("infoMessages", org.apache.thrift.protocol.TType.LIST, (short)2); - private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); - private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); - private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStatusStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStatusTupleSchemeFactory()); - } - - private TStatusCode statusCode; // required - private List infoMessages; // optional - private String sqlState; // optional - private int errorCode; // optional - private String errorMessage; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - /** - * - * @see TStatusCode - */ - STATUS_CODE((short)1, "statusCode"), - INFO_MESSAGES((short)2, "infoMessages"), - SQL_STATE((short)3, "sqlState"), - ERROR_CODE((short)4, "errorCode"), - ERROR_MESSAGE((short)5, "errorMessage"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATUS_CODE - return STATUS_CODE; - case 2: // INFO_MESSAGES - return INFO_MESSAGES; - case 3: // SQL_STATE - return SQL_STATE; - case 4: // ERROR_CODE - return ERROR_CODE; - case 5: // ERROR_MESSAGE - return ERROR_MESSAGE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ERRORCODE_ISSET_ID = 0; - private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.INFO_MESSAGES,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATUS_CODE, new org.apache.thrift.meta_data.FieldMetaData("statusCode", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TStatusCode.class))); - tmpMap.put(_Fields.INFO_MESSAGES, new org.apache.thrift.meta_data.FieldMetaData("infoMessages", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStatus.class, metaDataMap); - } - - public TStatus() { - } - - public TStatus( - TStatusCode statusCode) - { - this(); - this.statusCode = statusCode; - } - - /** - * Performs a deep copy on other. - */ - public TStatus(TStatus other) { - __isset_bitfield = other.__isset_bitfield; - if (other.isSetStatusCode()) { - this.statusCode = other.statusCode; - } - if (other.isSetInfoMessages()) { - List __this__infoMessages = new ArrayList(); - for (String other_element : other.infoMessages) { - __this__infoMessages.add(other_element); - } - this.infoMessages = __this__infoMessages; - } - if (other.isSetSqlState()) { - this.sqlState = other.sqlState; - } - this.errorCode = other.errorCode; - if (other.isSetErrorMessage()) { - this.errorMessage = other.errorMessage; - } - } - - public TStatus deepCopy() { - return new TStatus(this); - } - - @Override - public void clear() { - this.statusCode = null; - this.infoMessages = null; - this.sqlState = null; - setErrorCodeIsSet(false); - this.errorCode = 0; - this.errorMessage = null; - } - - /** - * - * @see TStatusCode - */ - public TStatusCode getStatusCode() { - return this.statusCode; - } - - /** - * - * @see TStatusCode - */ - public void setStatusCode(TStatusCode statusCode) { - this.statusCode = statusCode; - } - - public void unsetStatusCode() { - this.statusCode = null; - } - - /** Returns true if field statusCode is set (has been assigned a value) and false otherwise */ - public boolean isSetStatusCode() { - return this.statusCode != null; - } - - public void setStatusCodeIsSet(boolean value) { - if (!value) { - this.statusCode = null; - } - } - - public int getInfoMessagesSize() { - return (this.infoMessages == null) ? 0 : this.infoMessages.size(); - } - - public java.util.Iterator getInfoMessagesIterator() { - return (this.infoMessages == null) ? null : this.infoMessages.iterator(); - } - - public void addToInfoMessages(String elem) { - if (this.infoMessages == null) { - this.infoMessages = new ArrayList(); - } - this.infoMessages.add(elem); - } - - public List getInfoMessages() { - return this.infoMessages; - } - - public void setInfoMessages(List infoMessages) { - this.infoMessages = infoMessages; - } - - public void unsetInfoMessages() { - this.infoMessages = null; - } - - /** Returns true if field infoMessages is set (has been assigned a value) and false otherwise */ - public boolean isSetInfoMessages() { - return this.infoMessages != null; - } - - public void setInfoMessagesIsSet(boolean value) { - if (!value) { - this.infoMessages = null; - } - } - - public String getSqlState() { - return this.sqlState; - } - - public void setSqlState(String sqlState) { - this.sqlState = sqlState; - } - - public void unsetSqlState() { - this.sqlState = null; - } - - /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ - public boolean isSetSqlState() { - return this.sqlState != null; - } - - public void setSqlStateIsSet(boolean value) { - if (!value) { - this.sqlState = null; - } - } - - public int getErrorCode() { - return this.errorCode; - } - - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - setErrorCodeIsSet(true); - } - - public void unsetErrorCode() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorCode() { - return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); - } - - public void setErrorCodeIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); - } - - public String getErrorMessage() { - return this.errorMessage; - } - - public void setErrorMessage(String errorMessage) { - this.errorMessage = errorMessage; - } - - public void unsetErrorMessage() { - this.errorMessage = null; - } - - /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ - public boolean isSetErrorMessage() { - return this.errorMessage != null; - } - - public void setErrorMessageIsSet(boolean value) { - if (!value) { - this.errorMessage = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATUS_CODE: - if (value == null) { - unsetStatusCode(); - } else { - setStatusCode((TStatusCode)value); - } - break; - - case INFO_MESSAGES: - if (value == null) { - unsetInfoMessages(); - } else { - setInfoMessages((List)value); - } - break; - - case SQL_STATE: - if (value == null) { - unsetSqlState(); - } else { - setSqlState((String)value); - } - break; - - case ERROR_CODE: - if (value == null) { - unsetErrorCode(); - } else { - setErrorCode((Integer)value); - } - break; - - case ERROR_MESSAGE: - if (value == null) { - unsetErrorMessage(); - } else { - setErrorMessage((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATUS_CODE: - return getStatusCode(); - - case INFO_MESSAGES: - return getInfoMessages(); - - case SQL_STATE: - return getSqlState(); - - case ERROR_CODE: - return Integer.valueOf(getErrorCode()); - - case ERROR_MESSAGE: - return getErrorMessage(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATUS_CODE: - return isSetStatusCode(); - case INFO_MESSAGES: - return isSetInfoMessages(); - case SQL_STATE: - return isSetSqlState(); - case ERROR_CODE: - return isSetErrorCode(); - case ERROR_MESSAGE: - return isSetErrorMessage(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStatus) - return this.equals((TStatus)that); - return false; - } - - public boolean equals(TStatus that) { - if (that == null) - return false; - - boolean this_present_statusCode = true && this.isSetStatusCode(); - boolean that_present_statusCode = true && that.isSetStatusCode(); - if (this_present_statusCode || that_present_statusCode) { - if (!(this_present_statusCode && that_present_statusCode)) - return false; - if (!this.statusCode.equals(that.statusCode)) - return false; - } - - boolean this_present_infoMessages = true && this.isSetInfoMessages(); - boolean that_present_infoMessages = true && that.isSetInfoMessages(); - if (this_present_infoMessages || that_present_infoMessages) { - if (!(this_present_infoMessages && that_present_infoMessages)) - return false; - if (!this.infoMessages.equals(that.infoMessages)) - return false; - } - - boolean this_present_sqlState = true && this.isSetSqlState(); - boolean that_present_sqlState = true && that.isSetSqlState(); - if (this_present_sqlState || that_present_sqlState) { - if (!(this_present_sqlState && that_present_sqlState)) - return false; - if (!this.sqlState.equals(that.sqlState)) - return false; - } - - boolean this_present_errorCode = true && this.isSetErrorCode(); - boolean that_present_errorCode = true && that.isSetErrorCode(); - if (this_present_errorCode || that_present_errorCode) { - if (!(this_present_errorCode && that_present_errorCode)) - return false; - if (this.errorCode != that.errorCode) - return false; - } - - boolean this_present_errorMessage = true && this.isSetErrorMessage(); - boolean that_present_errorMessage = true && that.isSetErrorMessage(); - if (this_present_errorMessage || that_present_errorMessage) { - if (!(this_present_errorMessage && that_present_errorMessage)) - return false; - if (!this.errorMessage.equals(that.errorMessage)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_statusCode = true && (isSetStatusCode()); - builder.append(present_statusCode); - if (present_statusCode) - builder.append(statusCode.getValue()); - - boolean present_infoMessages = true && (isSetInfoMessages()); - builder.append(present_infoMessages); - if (present_infoMessages) - builder.append(infoMessages); - - boolean present_sqlState = true && (isSetSqlState()); - builder.append(present_sqlState); - if (present_sqlState) - builder.append(sqlState); - - boolean present_errorCode = true && (isSetErrorCode()); - builder.append(present_errorCode); - if (present_errorCode) - builder.append(errorCode); - - boolean present_errorMessage = true && (isSetErrorMessage()); - builder.append(present_errorMessage); - if (present_errorMessage) - builder.append(errorMessage); - - return builder.toHashCode(); - } - - public int compareTo(TStatus other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStatus typedOther = (TStatus)other; - - lastComparison = Boolean.valueOf(isSetStatusCode()).compareTo(typedOther.isSetStatusCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStatusCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statusCode, typedOther.statusCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetInfoMessages()).compareTo(typedOther.isSetInfoMessages()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetInfoMessages()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoMessages, typedOther.infoMessages); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSqlState()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorCode()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetErrorMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStatus("); - boolean first = true; - - sb.append("statusCode:"); - if (this.statusCode == null) { - sb.append("null"); - } else { - sb.append(this.statusCode); - } - first = false; - if (isSetInfoMessages()) { - if (!first) sb.append(", "); - sb.append("infoMessages:"); - if (this.infoMessages == null) { - sb.append("null"); - } else { - sb.append(this.infoMessages); - } - first = false; - } - if (isSetSqlState()) { - if (!first) sb.append(", "); - sb.append("sqlState:"); - if (this.sqlState == null) { - sb.append("null"); - } else { - sb.append(this.sqlState); - } - first = false; - } - if (isSetErrorCode()) { - if (!first) sb.append(", "); - sb.append("errorCode:"); - sb.append(this.errorCode); - first = false; - } - if (isSetErrorMessage()) { - if (!first) sb.append(", "); - sb.append("errorMessage:"); - if (this.errorMessage == null) { - sb.append("null"); - } else { - sb.append(this.errorMessage); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetStatusCode()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'statusCode' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStatusStandardSchemeFactory implements SchemeFactory { - public TStatusStandardScheme getScheme() { - return new TStatusStandardScheme(); - } - } - - private static class TStatusStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStatus struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STATUS_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.statusCode = TStatusCode.findByValue(iprot.readI32()); - struct.setStatusCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // INFO_MESSAGES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); - struct.infoMessages = new ArrayList(_list134.size); - for (int _i135 = 0; _i135 < _list134.size; ++_i135) - { - String _elem136; // optional - _elem136 = iprot.readString(); - struct.infoMessages.add(_elem136); - } - iprot.readListEnd(); - } - struct.setInfoMessagesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // SQL_STATE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // ERROR_CODE - if (schemeField.type == org.apache.thrift.protocol.TType.I32) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 5: // ERROR_MESSAGE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStatus struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.statusCode != null) { - oprot.writeFieldBegin(STATUS_CODE_FIELD_DESC); - oprot.writeI32(struct.statusCode.getValue()); - oprot.writeFieldEnd(); - } - if (struct.infoMessages != null) { - if (struct.isSetInfoMessages()) { - oprot.writeFieldBegin(INFO_MESSAGES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.infoMessages.size())); - for (String _iter137 : struct.infoMessages) - { - oprot.writeString(_iter137); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - if (struct.sqlState != null) { - if (struct.isSetSqlState()) { - oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); - oprot.writeString(struct.sqlState); - oprot.writeFieldEnd(); - } - } - if (struct.isSetErrorCode()) { - oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); - oprot.writeI32(struct.errorCode); - oprot.writeFieldEnd(); - } - if (struct.errorMessage != null) { - if (struct.isSetErrorMessage()) { - oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); - oprot.writeString(struct.errorMessage); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStatusTupleSchemeFactory implements SchemeFactory { - public TStatusTupleScheme getScheme() { - return new TStatusTupleScheme(); - } - } - - private static class TStatusTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeI32(struct.statusCode.getValue()); - BitSet optionals = new BitSet(); - if (struct.isSetInfoMessages()) { - optionals.set(0); - } - if (struct.isSetSqlState()) { - optionals.set(1); - } - if (struct.isSetErrorCode()) { - optionals.set(2); - } - if (struct.isSetErrorMessage()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); - if (struct.isSetInfoMessages()) { - { - oprot.writeI32(struct.infoMessages.size()); - for (String _iter138 : struct.infoMessages) - { - oprot.writeString(_iter138); - } - } - } - if (struct.isSetSqlState()) { - oprot.writeString(struct.sqlState); - } - if (struct.isSetErrorCode()) { - oprot.writeI32(struct.errorCode); - } - if (struct.isSetErrorMessage()) { - oprot.writeString(struct.errorMessage); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.statusCode = TStatusCode.findByValue(iprot.readI32()); - struct.setStatusCodeIsSet(true); - BitSet incoming = iprot.readBitSet(4); - if (incoming.get(0)) { - { - org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.infoMessages = new ArrayList(_list139.size); - for (int _i140 = 0; _i140 < _list139.size; ++_i140) - { - String _elem141; // optional - _elem141 = iprot.readString(); - struct.infoMessages.add(_elem141); - } - } - struct.setInfoMessagesIsSet(true); - } - if (incoming.get(1)) { - struct.sqlState = iprot.readString(); - struct.setSqlStateIsSet(true); - } - if (incoming.get(2)) { - struct.errorCode = iprot.readI32(); - struct.setErrorCodeIsSet(true); - } - if (incoming.get(3)) { - struct.errorMessage = iprot.readString(); - struct.setErrorMessageIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java deleted file mode 100644 index e7fde45fd131a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TStatusCode implements org.apache.thrift.TEnum { - SUCCESS_STATUS(0), - SUCCESS_WITH_INFO_STATUS(1), - STILL_EXECUTING_STATUS(2), - ERROR_STATUS(3), - INVALID_HANDLE_STATUS(4); - - private final int value; - - private TStatusCode(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TStatusCode findByValue(int value) { - switch (value) { - case 0: - return SUCCESS_STATUS; - case 1: - return SUCCESS_WITH_INFO_STATUS; - case 2: - return STILL_EXECUTING_STATUS; - case 3: - return ERROR_STATUS; - case 4: - return INVALID_HANDLE_STATUS; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java deleted file mode 100644 index 3dae460c8621d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java +++ /dev/null @@ -1,548 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStringColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringColumn"); - - private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); - private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStringColumnStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStringColumnTupleSchemeFactory()); - } - - private List values; // required - private ByteBuffer nulls; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUES((short)1, "values"), - NULLS((short)2, "nulls"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUES - return VALUES; - case 2: // NULLS - return NULLS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); - tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringColumn.class, metaDataMap); - } - - public TStringColumn() { - } - - public TStringColumn( - List values, - ByteBuffer nulls) - { - this(); - this.values = values; - this.nulls = nulls; - } - - /** - * Performs a deep copy on other. - */ - public TStringColumn(TStringColumn other) { - if (other.isSetValues()) { - List __this__values = new ArrayList(); - for (String other_element : other.values) { - __this__values.add(other_element); - } - this.values = __this__values; - } - if (other.isSetNulls()) { - this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); -; - } - } - - public TStringColumn deepCopy() { - return new TStringColumn(this); - } - - @Override - public void clear() { - this.values = null; - this.nulls = null; - } - - public int getValuesSize() { - return (this.values == null) ? 0 : this.values.size(); - } - - public java.util.Iterator getValuesIterator() { - return (this.values == null) ? null : this.values.iterator(); - } - - public void addToValues(String elem) { - if (this.values == null) { - this.values = new ArrayList(); - } - this.values.add(elem); - } - - public List getValues() { - return this.values; - } - - public void setValues(List values) { - this.values = values; - } - - public void unsetValues() { - this.values = null; - } - - /** Returns true if field values is set (has been assigned a value) and false otherwise */ - public boolean isSetValues() { - return this.values != null; - } - - public void setValuesIsSet(boolean value) { - if (!value) { - this.values = null; - } - } - - public byte[] getNulls() { - setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); - return nulls == null ? null : nulls.array(); - } - - public ByteBuffer bufferForNulls() { - return nulls; - } - - public void setNulls(byte[] nulls) { - setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); - } - - public void setNulls(ByteBuffer nulls) { - this.nulls = nulls; - } - - public void unsetNulls() { - this.nulls = null; - } - - /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ - public boolean isSetNulls() { - return this.nulls != null; - } - - public void setNullsIsSet(boolean value) { - if (!value) { - this.nulls = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUES: - if (value == null) { - unsetValues(); - } else { - setValues((List)value); - } - break; - - case NULLS: - if (value == null) { - unsetNulls(); - } else { - setNulls((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUES: - return getValues(); - - case NULLS: - return getNulls(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUES: - return isSetValues(); - case NULLS: - return isSetNulls(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStringColumn) - return this.equals((TStringColumn)that); - return false; - } - - public boolean equals(TStringColumn that) { - if (that == null) - return false; - - boolean this_present_values = true && this.isSetValues(); - boolean that_present_values = true && that.isSetValues(); - if (this_present_values || that_present_values) { - if (!(this_present_values && that_present_values)) - return false; - if (!this.values.equals(that.values)) - return false; - } - - boolean this_present_nulls = true && this.isSetNulls(); - boolean that_present_nulls = true && that.isSetNulls(); - if (this_present_nulls || that_present_nulls) { - if (!(this_present_nulls && that_present_nulls)) - return false; - if (!this.nulls.equals(that.nulls)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_values = true && (isSetValues()); - builder.append(present_values); - if (present_values) - builder.append(values); - - boolean present_nulls = true && (isSetNulls()); - builder.append(present_nulls); - if (present_nulls) - builder.append(nulls); - - return builder.toHashCode(); - } - - public int compareTo(TStringColumn other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStringColumn typedOther = (TStringColumn)other; - - lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNulls()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStringColumn("); - boolean first = true; - - sb.append("values:"); - if (this.values == null) { - sb.append("null"); - } else { - sb.append(this.values); - } - first = false; - if (!first) sb.append(", "); - sb.append("nulls:"); - if (this.nulls == null) { - sb.append("null"); - } else { - org.apache.thrift.TBaseHelper.toString(this.nulls, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetValues()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); - } - - if (!isSetNulls()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStringColumnStandardSchemeFactory implements SchemeFactory { - public TStringColumnStandardScheme getScheme() { - return new TStringColumnStandardScheme(); - } - } - - private static class TStringColumnStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStringColumn struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list102 = iprot.readListBegin(); - struct.values = new ArrayList(_list102.size); - for (int _i103 = 0; _i103 < _list102.size; ++_i103) - { - String _elem104; // optional - _elem104 = iprot.readString(); - struct.values.add(_elem104); - } - iprot.readListEnd(); - } - struct.setValuesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // NULLS - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStringColumn struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.values != null) { - oprot.writeFieldBegin(VALUES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); - for (String _iter105 : struct.values) - { - oprot.writeString(_iter105); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (struct.nulls != null) { - oprot.writeFieldBegin(NULLS_FIELD_DESC); - oprot.writeBinary(struct.nulls); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStringColumnTupleSchemeFactory implements SchemeFactory { - public TStringColumnTupleScheme getScheme() { - return new TStringColumnTupleScheme(); - } - } - - private static class TStringColumnTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.values.size()); - for (String _iter106 : struct.values) - { - oprot.writeString(_iter106); - } - } - oprot.writeBinary(struct.nulls); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.values = new ArrayList(_list107.size); - for (int _i108 = 0; _i108 < _list107.size; ++_i108) - { - String _elem109; // optional - _elem109 = iprot.readString(); - struct.values.add(_elem109); - } - } - struct.setValuesIsSet(true); - struct.nulls = iprot.readBinary(); - struct.setNullsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java deleted file mode 100644 index af7a109775a8b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java +++ /dev/null @@ -1,389 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStringValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringValue"); - - private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStringValueStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStringValueTupleSchemeFactory()); - } - - private String value; // optional - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - VALUE((short)1, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private _Fields optionals[] = {_Fields.VALUE}; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringValue.class, metaDataMap); - } - - public TStringValue() { - } - - /** - * Performs a deep copy on other. - */ - public TStringValue(TStringValue other) { - if (other.isSetValue()) { - this.value = other.value; - } - } - - public TStringValue deepCopy() { - return new TStringValue(this); - } - - @Override - public void clear() { - this.value = null; - } - - public String getValue() { - return this.value; - } - - public void setValue(String value) { - this.value = value; - } - - public void unsetValue() { - this.value = null; - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean isSetValue() { - return this.value != null; - } - - public void setValueIsSet(boolean value) { - if (!value) { - this.value = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case VALUE: - if (value == null) { - unsetValue(); - } else { - setValue((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case VALUE: - return getValue(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case VALUE: - return isSetValue(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStringValue) - return this.equals((TStringValue)that); - return false; - } - - public boolean equals(TStringValue that) { - if (that == null) - return false; - - boolean this_present_value = true && this.isSetValue(); - boolean that_present_value = true && that.isSetValue(); - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (!this.value.equals(that.value)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_value = true && (isSetValue()); - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(TStringValue other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStringValue typedOther = (TStringValue)other; - - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStringValue("); - boolean first = true; - - if (isSetValue()) { - sb.append("value:"); - if (this.value == null) { - sb.append("null"); - } else { - sb.append(this.value); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStringValueStandardSchemeFactory implements SchemeFactory { - public TStringValueStandardScheme getScheme() { - return new TStringValueStandardScheme(); - } - } - - private static class TStringValueStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStringValue struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // VALUE - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.value = iprot.readString(); - struct.setValueIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStringValue struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.value != null) { - if (struct.isSetValue()) { - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeString(struct.value); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStringValueTupleSchemeFactory implements SchemeFactory { - public TStringValueTupleScheme getScheme() { - return new TStringValueTupleScheme(); - } - } - - private static class TStringValueTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetValue()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetValue()) { - oprot.writeString(struct.value); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.value = iprot.readString(); - struct.setValueIsSet(true); - } - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java deleted file mode 100644 index 20f5fb6c29073..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java +++ /dev/null @@ -1,448 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TStructTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStructTypeEntry"); - - private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TStructTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TStructTypeEntryTupleSchemeFactory()); - } - - private Map nameToTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME_TO_TYPE_PTR - return NAME_TO_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStructTypeEntry.class, metaDataMap); - } - - public TStructTypeEntry() { - } - - public TStructTypeEntry( - Map nameToTypePtr) - { - this(); - this.nameToTypePtr = nameToTypePtr; - } - - /** - * Performs a deep copy on other. - */ - public TStructTypeEntry(TStructTypeEntry other) { - if (other.isSetNameToTypePtr()) { - Map __this__nameToTypePtr = new HashMap(); - for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { - - String other_element_key = other_element.getKey(); - Integer other_element_value = other_element.getValue(); - - String __this__nameToTypePtr_copy_key = other_element_key; - - Integer __this__nameToTypePtr_copy_value = other_element_value; - - __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); - } - this.nameToTypePtr = __this__nameToTypePtr; - } - } - - public TStructTypeEntry deepCopy() { - return new TStructTypeEntry(this); - } - - @Override - public void clear() { - this.nameToTypePtr = null; - } - - public int getNameToTypePtrSize() { - return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); - } - - public void putToNameToTypePtr(String key, int val) { - if (this.nameToTypePtr == null) { - this.nameToTypePtr = new HashMap(); - } - this.nameToTypePtr.put(key, val); - } - - public Map getNameToTypePtr() { - return this.nameToTypePtr; - } - - public void setNameToTypePtr(Map nameToTypePtr) { - this.nameToTypePtr = nameToTypePtr; - } - - public void unsetNameToTypePtr() { - this.nameToTypePtr = null; - } - - /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetNameToTypePtr() { - return this.nameToTypePtr != null; - } - - public void setNameToTypePtrIsSet(boolean value) { - if (!value) { - this.nameToTypePtr = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME_TO_TYPE_PTR: - if (value == null) { - unsetNameToTypePtr(); - } else { - setNameToTypePtr((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME_TO_TYPE_PTR: - return getNameToTypePtr(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME_TO_TYPE_PTR: - return isSetNameToTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TStructTypeEntry) - return this.equals((TStructTypeEntry)that); - return false; - } - - public boolean equals(TStructTypeEntry that) { - if (that == null) - return false; - - boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); - boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); - if (this_present_nameToTypePtr || that_present_nameToTypePtr) { - if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) - return false; - if (!this.nameToTypePtr.equals(that.nameToTypePtr)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); - builder.append(present_nameToTypePtr); - if (present_nameToTypePtr) - builder.append(nameToTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TStructTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TStructTypeEntry typedOther = (TStructTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNameToTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TStructTypeEntry("); - boolean first = true; - - sb.append("nameToTypePtr:"); - if (this.nameToTypePtr == null) { - sb.append("null"); - } else { - sb.append(this.nameToTypePtr); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetNameToTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TStructTypeEntryStandardSchemeFactory implements SchemeFactory { - public TStructTypeEntryStandardScheme getScheme() { - return new TStructTypeEntryStandardScheme(); - } - } - - private static class TStructTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TStructTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // NAME_TO_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map10 = iprot.readMapBegin(); - struct.nameToTypePtr = new HashMap(2*_map10.size); - for (int _i11 = 0; _i11 < _map10.size; ++_i11) - { - String _key12; // required - int _val13; // required - _key12 = iprot.readString(); - _val13 = iprot.readI32(); - struct.nameToTypePtr.put(_key12, _val13); - } - iprot.readMapEnd(); - } - struct.setNameToTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TStructTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.nameToTypePtr != null) { - oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); - for (Map.Entry _iter14 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter14.getKey()); - oprot.writeI32(_iter14.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TStructTypeEntryTupleSchemeFactory implements SchemeFactory { - public TStructTypeEntryTupleScheme getScheme() { - return new TStructTypeEntryTupleScheme(); - } - } - - private static class TStructTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.nameToTypePtr.size()); - for (Map.Entry _iter15 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter15.getKey()); - oprot.writeI32(_iter15.getValue()); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map16 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.nameToTypePtr = new HashMap(2*_map16.size); - for (int _i17 = 0; _i17 < _map16.size; ++_i17) - { - String _key18; // required - int _val19; // required - _key18 = iprot.readString(); - _val19 = iprot.readI32(); - struct.nameToTypePtr.put(_key18, _val19); - } - } - struct.setNameToTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java deleted file mode 100644 index ff5e54db7c16c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTableSchema implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableSchema"); - - private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTableSchemaStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTableSchemaTupleSchemeFactory()); - } - - private List columns; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - COLUMNS((short)1, "columns"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COLUMNS - return COLUMNS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnDesc.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableSchema.class, metaDataMap); - } - - public TTableSchema() { - } - - public TTableSchema( - List columns) - { - this(); - this.columns = columns; - } - - /** - * Performs a deep copy on other. - */ - public TTableSchema(TTableSchema other) { - if (other.isSetColumns()) { - List __this__columns = new ArrayList(); - for (TColumnDesc other_element : other.columns) { - __this__columns.add(new TColumnDesc(other_element)); - } - this.columns = __this__columns; - } - } - - public TTableSchema deepCopy() { - return new TTableSchema(this); - } - - @Override - public void clear() { - this.columns = null; - } - - public int getColumnsSize() { - return (this.columns == null) ? 0 : this.columns.size(); - } - - public java.util.Iterator getColumnsIterator() { - return (this.columns == null) ? null : this.columns.iterator(); - } - - public void addToColumns(TColumnDesc elem) { - if (this.columns == null) { - this.columns = new ArrayList(); - } - this.columns.add(elem); - } - - public List getColumns() { - return this.columns; - } - - public void setColumns(List columns) { - this.columns = columns; - } - - public void unsetColumns() { - this.columns = null; - } - - /** Returns true if field columns is set (has been assigned a value) and false otherwise */ - public boolean isSetColumns() { - return this.columns != null; - } - - public void setColumnsIsSet(boolean value) { - if (!value) { - this.columns = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COLUMNS: - if (value == null) { - unsetColumns(); - } else { - setColumns((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COLUMNS: - return getColumns(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COLUMNS: - return isSetColumns(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTableSchema) - return this.equals((TTableSchema)that); - return false; - } - - public boolean equals(TTableSchema that) { - if (that == null) - return false; - - boolean this_present_columns = true && this.isSetColumns(); - boolean that_present_columns = true && that.isSetColumns(); - if (this_present_columns || that_present_columns) { - if (!(this_present_columns && that_present_columns)) - return false; - if (!this.columns.equals(that.columns)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_columns = true && (isSetColumns()); - builder.append(present_columns); - if (present_columns) - builder.append(columns); - - return builder.toHashCode(); - } - - public int compareTo(TTableSchema other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTableSchema typedOther = (TTableSchema)other; - - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTableSchema("); - boolean first = true; - - sb.append("columns:"); - if (this.columns == null) { - sb.append("null"); - } else { - sb.append(this.columns); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetColumns()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTableSchemaStandardSchemeFactory implements SchemeFactory { - public TTableSchemaStandardScheme getScheme() { - return new TTableSchemaStandardScheme(); - } - } - - private static class TTableSchemaStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTableSchema struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // COLUMNS - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list38 = iprot.readListBegin(); - struct.columns = new ArrayList(_list38.size); - for (int _i39 = 0; _i39 < _list38.size; ++_i39) - { - TColumnDesc _elem40; // optional - _elem40 = new TColumnDesc(); - _elem40.read(iprot); - struct.columns.add(_elem40); - } - iprot.readListEnd(); - } - struct.setColumnsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTableSchema struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.columns != null) { - oprot.writeFieldBegin(COLUMNS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); - for (TColumnDesc _iter41 : struct.columns) - { - _iter41.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTableSchemaTupleSchemeFactory implements SchemeFactory { - public TTableSchemaTupleScheme getScheme() { - return new TTableSchemaTupleScheme(); - } - } - - private static class TTableSchemaTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.columns.size()); - for (TColumnDesc _iter42 : struct.columns) - { - _iter42.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list43 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columns = new ArrayList(_list43.size); - for (int _i44 = 0; _i44 < _list43.size; ++_i44) - { - TColumnDesc _elem45; // optional - _elem45 = new TColumnDesc(); - _elem45.read(iprot); - struct.columns.add(_elem45); - } - } - struct.setColumnsIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java deleted file mode 100644 index 251f86a914719..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java +++ /dev/null @@ -1,439 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeDesc"); - - private static final org.apache.thrift.protocol.TField TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("types", org.apache.thrift.protocol.TType.LIST, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTypeDescStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTypeDescTupleSchemeFactory()); - } - - private List types; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - TYPES((short)1, "types"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPES - return TYPES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPES, new org.apache.thrift.meta_data.FieldMetaData("types", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeEntry.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeDesc.class, metaDataMap); - } - - public TTypeDesc() { - } - - public TTypeDesc( - List types) - { - this(); - this.types = types; - } - - /** - * Performs a deep copy on other. - */ - public TTypeDesc(TTypeDesc other) { - if (other.isSetTypes()) { - List __this__types = new ArrayList(); - for (TTypeEntry other_element : other.types) { - __this__types.add(new TTypeEntry(other_element)); - } - this.types = __this__types; - } - } - - public TTypeDesc deepCopy() { - return new TTypeDesc(this); - } - - @Override - public void clear() { - this.types = null; - } - - public int getTypesSize() { - return (this.types == null) ? 0 : this.types.size(); - } - - public java.util.Iterator getTypesIterator() { - return (this.types == null) ? null : this.types.iterator(); - } - - public void addToTypes(TTypeEntry elem) { - if (this.types == null) { - this.types = new ArrayList(); - } - this.types.add(elem); - } - - public List getTypes() { - return this.types; - } - - public void setTypes(List types) { - this.types = types; - } - - public void unsetTypes() { - this.types = null; - } - - /** Returns true if field types is set (has been assigned a value) and false otherwise */ - public boolean isSetTypes() { - return this.types != null; - } - - public void setTypesIsSet(boolean value) { - if (!value) { - this.types = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPES: - if (value == null) { - unsetTypes(); - } else { - setTypes((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPES: - return getTypes(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPES: - return isSetTypes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTypeDesc) - return this.equals((TTypeDesc)that); - return false; - } - - public boolean equals(TTypeDesc that) { - if (that == null) - return false; - - boolean this_present_types = true && this.isSetTypes(); - boolean that_present_types = true && that.isSetTypes(); - if (this_present_types || that_present_types) { - if (!(this_present_types && that_present_types)) - return false; - if (!this.types.equals(that.types)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_types = true && (isSetTypes()); - builder.append(present_types); - if (present_types) - builder.append(types); - - return builder.toHashCode(); - } - - public int compareTo(TTypeDesc other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTypeDesc typedOther = (TTypeDesc)other; - - lastComparison = Boolean.valueOf(isSetTypes()).compareTo(typedOther.isSetTypes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.types, typedOther.types); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTypeDesc("); - boolean first = true; - - sb.append("types:"); - if (this.types == null) { - sb.append("null"); - } else { - sb.append(this.types); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetTypes()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'types' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTypeDescStandardSchemeFactory implements SchemeFactory { - public TTypeDescStandardScheme getScheme() { - return new TTypeDescStandardScheme(); - } - } - - private static class TTypeDescStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeDesc struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPES - if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { - { - org.apache.thrift.protocol.TList _list30 = iprot.readListBegin(); - struct.types = new ArrayList(_list30.size); - for (int _i31 = 0; _i31 < _list30.size; ++_i31) - { - TTypeEntry _elem32; // optional - _elem32 = new TTypeEntry(); - _elem32.read(iprot); - struct.types.add(_elem32); - } - iprot.readListEnd(); - } - struct.setTypesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeDesc struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.types != null) { - oprot.writeFieldBegin(TYPES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.types.size())); - for (TTypeEntry _iter33 : struct.types) - { - _iter33.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTypeDescTupleSchemeFactory implements SchemeFactory { - public TTypeDescTupleScheme getScheme() { - return new TTypeDescTupleScheme(); - } - } - - private static class TTypeDescTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.types.size()); - for (TTypeEntry _iter34 : struct.types) - { - _iter34.write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.types = new ArrayList(_list35.size); - for (int _i36 = 0; _i36 < _list35.size; ++_i36) - { - TTypeEntry _elem37; // optional - _elem37 = new TTypeEntry(); - _elem37.read(iprot); - struct.types.add(_elem37); - } - } - struct.setTypesIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java deleted file mode 100644 index d0d70c1279572..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java +++ /dev/null @@ -1,610 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeEntry extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeEntry"); - private static final org.apache.thrift.protocol.TField PRIMITIVE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("primitiveEntry", org.apache.thrift.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift.protocol.TField ARRAY_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("arrayEntry", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField MAP_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("mapEntry", org.apache.thrift.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift.protocol.TField STRUCT_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("structEntry", org.apache.thrift.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift.protocol.TField UNION_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("unionEntry", org.apache.thrift.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift.protocol.TField USER_DEFINED_TYPE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("userDefinedTypeEntry", org.apache.thrift.protocol.TType.STRUCT, (short)6); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - PRIMITIVE_ENTRY((short)1, "primitiveEntry"), - ARRAY_ENTRY((short)2, "arrayEntry"), - MAP_ENTRY((short)3, "mapEntry"), - STRUCT_ENTRY((short)4, "structEntry"), - UNION_ENTRY((short)5, "unionEntry"), - USER_DEFINED_TYPE_ENTRY((short)6, "userDefinedTypeEntry"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // PRIMITIVE_ENTRY - return PRIMITIVE_ENTRY; - case 2: // ARRAY_ENTRY - return ARRAY_ENTRY; - case 3: // MAP_ENTRY - return MAP_ENTRY; - case 4: // STRUCT_ENTRY - return STRUCT_ENTRY; - case 5: // UNION_ENTRY - return UNION_ENTRY; - case 6: // USER_DEFINED_TYPE_ENTRY - return USER_DEFINED_TYPE_ENTRY; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.PRIMITIVE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("primitiveEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TPrimitiveTypeEntry.class))); - tmpMap.put(_Fields.ARRAY_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("arrayEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TArrayTypeEntry.class))); - tmpMap.put(_Fields.MAP_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("mapEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TMapTypeEntry.class))); - tmpMap.put(_Fields.STRUCT_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("structEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStructTypeEntry.class))); - tmpMap.put(_Fields.UNION_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("unionEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUnionTypeEntry.class))); - tmpMap.put(_Fields.USER_DEFINED_TYPE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("userDefinedTypeEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUserDefinedTypeEntry.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeEntry.class, metaDataMap); - } - - public TTypeEntry() { - super(); - } - - public TTypeEntry(TTypeEntry._Fields setField, Object value) { - super(setField, value); - } - - public TTypeEntry(TTypeEntry other) { - super(other); - } - public TTypeEntry deepCopy() { - return new TTypeEntry(this); - } - - public static TTypeEntry primitiveEntry(TPrimitiveTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setPrimitiveEntry(value); - return x; - } - - public static TTypeEntry arrayEntry(TArrayTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setArrayEntry(value); - return x; - } - - public static TTypeEntry mapEntry(TMapTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setMapEntry(value); - return x; - } - - public static TTypeEntry structEntry(TStructTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setStructEntry(value); - return x; - } - - public static TTypeEntry unionEntry(TUnionTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setUnionEntry(value); - return x; - } - - public static TTypeEntry userDefinedTypeEntry(TUserDefinedTypeEntry value) { - TTypeEntry x = new TTypeEntry(); - x.setUserDefinedTypeEntry(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case PRIMITIVE_ENTRY: - if (value instanceof TPrimitiveTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TPrimitiveTypeEntry for field 'primitiveEntry', but got " + value.getClass().getSimpleName()); - case ARRAY_ENTRY: - if (value instanceof TArrayTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TArrayTypeEntry for field 'arrayEntry', but got " + value.getClass().getSimpleName()); - case MAP_ENTRY: - if (value instanceof TMapTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TMapTypeEntry for field 'mapEntry', but got " + value.getClass().getSimpleName()); - case STRUCT_ENTRY: - if (value instanceof TStructTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TStructTypeEntry for field 'structEntry', but got " + value.getClass().getSimpleName()); - case UNION_ENTRY: - if (value instanceof TUnionTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TUnionTypeEntry for field 'unionEntry', but got " + value.getClass().getSimpleName()); - case USER_DEFINED_TYPE_ENTRY: - if (value instanceof TUserDefinedTypeEntry) { - break; - } - throw new ClassCastException("Was expecting value of type TUserDefinedTypeEntry for field 'userDefinedTypeEntry', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case PRIMITIVE_ENTRY: - if (field.type == PRIMITIVE_ENTRY_FIELD_DESC.type) { - TPrimitiveTypeEntry primitiveEntry; - primitiveEntry = new TPrimitiveTypeEntry(); - primitiveEntry.read(iprot); - return primitiveEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case ARRAY_ENTRY: - if (field.type == ARRAY_ENTRY_FIELD_DESC.type) { - TArrayTypeEntry arrayEntry; - arrayEntry = new TArrayTypeEntry(); - arrayEntry.read(iprot); - return arrayEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case MAP_ENTRY: - if (field.type == MAP_ENTRY_FIELD_DESC.type) { - TMapTypeEntry mapEntry; - mapEntry = new TMapTypeEntry(); - mapEntry.read(iprot); - return mapEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRUCT_ENTRY: - if (field.type == STRUCT_ENTRY_FIELD_DESC.type) { - TStructTypeEntry structEntry; - structEntry = new TStructTypeEntry(); - structEntry.read(iprot); - return structEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case UNION_ENTRY: - if (field.type == UNION_ENTRY_FIELD_DESC.type) { - TUnionTypeEntry unionEntry; - unionEntry = new TUnionTypeEntry(); - unionEntry.read(iprot); - return unionEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case USER_DEFINED_TYPE_ENTRY: - if (field.type == USER_DEFINED_TYPE_ENTRY_FIELD_DESC.type) { - TUserDefinedTypeEntry userDefinedTypeEntry; - userDefinedTypeEntry = new TUserDefinedTypeEntry(); - userDefinedTypeEntry.read(iprot); - return userDefinedTypeEntry; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; - primitiveEntry.write(oprot); - return; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; - arrayEntry.write(oprot); - return; - case MAP_ENTRY: - TMapTypeEntry mapEntry = (TMapTypeEntry)value_; - mapEntry.write(oprot); - return; - case STRUCT_ENTRY: - TStructTypeEntry structEntry = (TStructTypeEntry)value_; - structEntry.write(oprot); - return; - case UNION_ENTRY: - TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; - unionEntry.write(oprot); - return; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; - userDefinedTypeEntry.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry; - primitiveEntry = new TPrimitiveTypeEntry(); - primitiveEntry.read(iprot); - return primitiveEntry; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry; - arrayEntry = new TArrayTypeEntry(); - arrayEntry.read(iprot); - return arrayEntry; - case MAP_ENTRY: - TMapTypeEntry mapEntry; - mapEntry = new TMapTypeEntry(); - mapEntry.read(iprot); - return mapEntry; - case STRUCT_ENTRY: - TStructTypeEntry structEntry; - structEntry = new TStructTypeEntry(); - structEntry.read(iprot); - return structEntry; - case UNION_ENTRY: - TUnionTypeEntry unionEntry; - unionEntry = new TUnionTypeEntry(); - unionEntry.read(iprot); - return unionEntry; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry; - userDefinedTypeEntry = new TUserDefinedTypeEntry(); - userDefinedTypeEntry.read(iprot); - return userDefinedTypeEntry; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case PRIMITIVE_ENTRY: - TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; - primitiveEntry.write(oprot); - return; - case ARRAY_ENTRY: - TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; - arrayEntry.write(oprot); - return; - case MAP_ENTRY: - TMapTypeEntry mapEntry = (TMapTypeEntry)value_; - mapEntry.write(oprot); - return; - case STRUCT_ENTRY: - TStructTypeEntry structEntry = (TStructTypeEntry)value_; - structEntry.write(oprot); - return; - case UNION_ENTRY: - TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; - unionEntry.write(oprot); - return; - case USER_DEFINED_TYPE_ENTRY: - TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; - userDefinedTypeEntry.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case PRIMITIVE_ENTRY: - return PRIMITIVE_ENTRY_FIELD_DESC; - case ARRAY_ENTRY: - return ARRAY_ENTRY_FIELD_DESC; - case MAP_ENTRY: - return MAP_ENTRY_FIELD_DESC; - case STRUCT_ENTRY: - return STRUCT_ENTRY_FIELD_DESC; - case UNION_ENTRY: - return UNION_ENTRY_FIELD_DESC; - case USER_DEFINED_TYPE_ENTRY: - return USER_DEFINED_TYPE_ENTRY_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public TPrimitiveTypeEntry getPrimitiveEntry() { - if (getSetField() == _Fields.PRIMITIVE_ENTRY) { - return (TPrimitiveTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'primitiveEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setPrimitiveEntry(TPrimitiveTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.PRIMITIVE_ENTRY; - value_ = value; - } - - public TArrayTypeEntry getArrayEntry() { - if (getSetField() == _Fields.ARRAY_ENTRY) { - return (TArrayTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'arrayEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setArrayEntry(TArrayTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.ARRAY_ENTRY; - value_ = value; - } - - public TMapTypeEntry getMapEntry() { - if (getSetField() == _Fields.MAP_ENTRY) { - return (TMapTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'mapEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setMapEntry(TMapTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.MAP_ENTRY; - value_ = value; - } - - public TStructTypeEntry getStructEntry() { - if (getSetField() == _Fields.STRUCT_ENTRY) { - return (TStructTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'structEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStructEntry(TStructTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRUCT_ENTRY; - value_ = value; - } - - public TUnionTypeEntry getUnionEntry() { - if (getSetField() == _Fields.UNION_ENTRY) { - return (TUnionTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'unionEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setUnionEntry(TUnionTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.UNION_ENTRY; - value_ = value; - } - - public TUserDefinedTypeEntry getUserDefinedTypeEntry() { - if (getSetField() == _Fields.USER_DEFINED_TYPE_ENTRY) { - return (TUserDefinedTypeEntry)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'userDefinedTypeEntry' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setUserDefinedTypeEntry(TUserDefinedTypeEntry value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.USER_DEFINED_TYPE_ENTRY; - value_ = value; - } - - public boolean isSetPrimitiveEntry() { - return setField_ == _Fields.PRIMITIVE_ENTRY; - } - - - public boolean isSetArrayEntry() { - return setField_ == _Fields.ARRAY_ENTRY; - } - - - public boolean isSetMapEntry() { - return setField_ == _Fields.MAP_ENTRY; - } - - - public boolean isSetStructEntry() { - return setField_ == _Fields.STRUCT_ENTRY; - } - - - public boolean isSetUnionEntry() { - return setField_ == _Fields.UNION_ENTRY; - } - - - public boolean isSetUserDefinedTypeEntry() { - return setField_ == _Fields.USER_DEFINED_TYPE_ENTRY; - } - - - public boolean equals(Object other) { - if (other instanceof TTypeEntry) { - return equals((TTypeEntry)other); - } else { - return false; - } - } - - public boolean equals(TTypeEntry other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TTypeEntry other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java deleted file mode 100644 index 40f05894623c0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - -public enum TTypeId implements org.apache.thrift.TEnum { - BOOLEAN_TYPE(0), - TINYINT_TYPE(1), - SMALLINT_TYPE(2), - INT_TYPE(3), - BIGINT_TYPE(4), - FLOAT_TYPE(5), - DOUBLE_TYPE(6), - STRING_TYPE(7), - TIMESTAMP_TYPE(8), - BINARY_TYPE(9), - ARRAY_TYPE(10), - MAP_TYPE(11), - STRUCT_TYPE(12), - UNION_TYPE(13), - USER_DEFINED_TYPE(14), - DECIMAL_TYPE(15), - NULL_TYPE(16), - DATE_TYPE(17), - VARCHAR_TYPE(18), - CHAR_TYPE(19), - INTERVAL_YEAR_MONTH_TYPE(20), - INTERVAL_DAY_TIME_TYPE(21); - - private final int value; - - private TTypeId(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static TTypeId findByValue(int value) { - switch (value) { - case 0: - return BOOLEAN_TYPE; - case 1: - return TINYINT_TYPE; - case 2: - return SMALLINT_TYPE; - case 3: - return INT_TYPE; - case 4: - return BIGINT_TYPE; - case 5: - return FLOAT_TYPE; - case 6: - return DOUBLE_TYPE; - case 7: - return STRING_TYPE; - case 8: - return TIMESTAMP_TYPE; - case 9: - return BINARY_TYPE; - case 10: - return ARRAY_TYPE; - case 11: - return MAP_TYPE; - case 12: - return STRUCT_TYPE; - case 13: - return UNION_TYPE; - case 14: - return USER_DEFINED_TYPE; - case 15: - return DECIMAL_TYPE; - case 16: - return NULL_TYPE; - case 17: - return DATE_TYPE; - case 18: - return VARCHAR_TYPE; - case 19: - return CHAR_TYPE; - case 20: - return INTERVAL_YEAR_MONTH_TYPE; - case 21: - return INTERVAL_DAY_TIME_TYPE; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java deleted file mode 100644 index a3e3829372276..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java +++ /dev/null @@ -1,361 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeQualifierValue extends org.apache.thrift.TUnion { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifierValue"); - private static final org.apache.thrift.protocol.TField I32_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Value", org.apache.thrift.protocol.TType.I32, (short)1); - private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)2); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - I32_VALUE((short)1, "i32Value"), - STRING_VALUE((short)2, "stringValue"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // I32_VALUE - return I32_VALUE; - case 2: // STRING_VALUE - return STRING_VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.I32_VALUE, new org.apache.thrift.meta_data.FieldMetaData("i32Value", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); - tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifierValue.class, metaDataMap); - } - - public TTypeQualifierValue() { - super(); - } - - public TTypeQualifierValue(TTypeQualifierValue._Fields setField, Object value) { - super(setField, value); - } - - public TTypeQualifierValue(TTypeQualifierValue other) { - super(other); - } - public TTypeQualifierValue deepCopy() { - return new TTypeQualifierValue(this); - } - - public static TTypeQualifierValue i32Value(int value) { - TTypeQualifierValue x = new TTypeQualifierValue(); - x.setI32Value(value); - return x; - } - - public static TTypeQualifierValue stringValue(String value) { - TTypeQualifierValue x = new TTypeQualifierValue(); - x.setStringValue(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case I32_VALUE: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'i32Value', but got " + value.getClass().getSimpleName()); - case STRING_VALUE: - if (value instanceof String) { - break; - } - throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case I32_VALUE: - if (field.type == I32_VALUE_FIELD_DESC.type) { - Integer i32Value; - i32Value = iprot.readI32(); - return i32Value; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_VALUE: - if (field.type == STRING_VALUE_FIELD_DESC.type) { - String stringValue; - stringValue = iprot.readString(); - return stringValue; - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - return null; - } - } - - @Override - protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case I32_VALUE: - Integer i32Value = (Integer)value_; - oprot.writeI32(i32Value); - return; - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { - _Fields setField = _Fields.findByThriftId(fieldID); - if (setField != null) { - switch (setField) { - case I32_VALUE: - Integer i32Value; - i32Value = iprot.readI32(); - return i32Value; - case STRING_VALUE: - String stringValue; - stringValue = iprot.readString(); - return stringValue; - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - throw new TProtocolException("Couldn't find a field with field id " + fieldID); - } - } - - @Override - protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - switch (setField_) { - case I32_VALUE: - Integer i32Value = (Integer)value_; - oprot.writeI32(i32Value); - return; - case STRING_VALUE: - String stringValue = (String)value_; - oprot.writeString(stringValue); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case I32_VALUE: - return I32_VALUE_FIELD_DESC; - case STRING_VALUE: - return STRING_VALUE_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public int getI32Value() { - if (getSetField() == _Fields.I32_VALUE) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'i32Value' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setI32Value(int value) { - setField_ = _Fields.I32_VALUE; - value_ = value; - } - - public String getStringValue() { - if (getSetField() == _Fields.STRING_VALUE) { - return (String)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void setStringValue(String value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_VALUE; - value_ = value; - } - - public boolean isSetI32Value() { - return setField_ == _Fields.I32_VALUE; - } - - - public boolean isSetStringValue() { - return setField_ == _Fields.STRING_VALUE; - } - - - public boolean equals(Object other) { - if (other instanceof TTypeQualifierValue) { - return equals((TTypeQualifierValue)other); - } else { - return false; - } - } - - public boolean equals(TTypeQualifierValue other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(TTypeQualifierValue other) { - int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift.TEnum) { - hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java deleted file mode 100644 index 39355551d3722..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java +++ /dev/null @@ -1,450 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TTypeQualifiers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifiers"); - - private static final org.apache.thrift.protocol.TField QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifiers", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TTypeQualifiersStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TTypeQualifiersTupleSchemeFactory()); - } - - private Map qualifiers; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - QUALIFIERS((short)1, "qualifiers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // QUALIFIERS - return QUALIFIERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("qualifiers", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifierValue.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifiers.class, metaDataMap); - } - - public TTypeQualifiers() { - } - - public TTypeQualifiers( - Map qualifiers) - { - this(); - this.qualifiers = qualifiers; - } - - /** - * Performs a deep copy on other. - */ - public TTypeQualifiers(TTypeQualifiers other) { - if (other.isSetQualifiers()) { - Map __this__qualifiers = new HashMap(); - for (Map.Entry other_element : other.qualifiers.entrySet()) { - - String other_element_key = other_element.getKey(); - TTypeQualifierValue other_element_value = other_element.getValue(); - - String __this__qualifiers_copy_key = other_element_key; - - TTypeQualifierValue __this__qualifiers_copy_value = new TTypeQualifierValue(other_element_value); - - __this__qualifiers.put(__this__qualifiers_copy_key, __this__qualifiers_copy_value); - } - this.qualifiers = __this__qualifiers; - } - } - - public TTypeQualifiers deepCopy() { - return new TTypeQualifiers(this); - } - - @Override - public void clear() { - this.qualifiers = null; - } - - public int getQualifiersSize() { - return (this.qualifiers == null) ? 0 : this.qualifiers.size(); - } - - public void putToQualifiers(String key, TTypeQualifierValue val) { - if (this.qualifiers == null) { - this.qualifiers = new HashMap(); - } - this.qualifiers.put(key, val); - } - - public Map getQualifiers() { - return this.qualifiers; - } - - public void setQualifiers(Map qualifiers) { - this.qualifiers = qualifiers; - } - - public void unsetQualifiers() { - this.qualifiers = null; - } - - /** Returns true if field qualifiers is set (has been assigned a value) and false otherwise */ - public boolean isSetQualifiers() { - return this.qualifiers != null; - } - - public void setQualifiersIsSet(boolean value) { - if (!value) { - this.qualifiers = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case QUALIFIERS: - if (value == null) { - unsetQualifiers(); - } else { - setQualifiers((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case QUALIFIERS: - return getQualifiers(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case QUALIFIERS: - return isSetQualifiers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TTypeQualifiers) - return this.equals((TTypeQualifiers)that); - return false; - } - - public boolean equals(TTypeQualifiers that) { - if (that == null) - return false; - - boolean this_present_qualifiers = true && this.isSetQualifiers(); - boolean that_present_qualifiers = true && that.isSetQualifiers(); - if (this_present_qualifiers || that_present_qualifiers) { - if (!(this_present_qualifiers && that_present_qualifiers)) - return false; - if (!this.qualifiers.equals(that.qualifiers)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_qualifiers = true && (isSetQualifiers()); - builder.append(present_qualifiers); - if (present_qualifiers) - builder.append(qualifiers); - - return builder.toHashCode(); - } - - public int compareTo(TTypeQualifiers other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TTypeQualifiers typedOther = (TTypeQualifiers)other; - - lastComparison = Boolean.valueOf(isSetQualifiers()).compareTo(typedOther.isSetQualifiers()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetQualifiers()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifiers, typedOther.qualifiers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TTypeQualifiers("); - boolean first = true; - - sb.append("qualifiers:"); - if (this.qualifiers == null) { - sb.append("null"); - } else { - sb.append(this.qualifiers); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetQualifiers()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifiers' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TTypeQualifiersStandardSchemeFactory implements SchemeFactory { - public TTypeQualifiersStandardScheme getScheme() { - return new TTypeQualifiersStandardScheme(); - } - } - - private static class TTypeQualifiersStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeQualifiers struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // QUALIFIERS - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin(); - struct.qualifiers = new HashMap(2*_map0.size); - for (int _i1 = 0; _i1 < _map0.size; ++_i1) - { - String _key2; // required - TTypeQualifierValue _val3; // required - _key2 = iprot.readString(); - _val3 = new TTypeQualifierValue(); - _val3.read(iprot); - struct.qualifiers.put(_key2, _val3); - } - iprot.readMapEnd(); - } - struct.setQualifiersIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeQualifiers struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.qualifiers != null) { - oprot.writeFieldBegin(QUALIFIERS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.qualifiers.size())); - for (Map.Entry _iter4 : struct.qualifiers.entrySet()) - { - oprot.writeString(_iter4.getKey()); - _iter4.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TTypeQualifiersTupleSchemeFactory implements SchemeFactory { - public TTypeQualifiersTupleScheme getScheme() { - return new TTypeQualifiersTupleScheme(); - } - } - - private static class TTypeQualifiersTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.qualifiers.size()); - for (Map.Entry _iter5 : struct.qualifiers.entrySet()) - { - oprot.writeString(_iter5.getKey()); - _iter5.getValue().write(oprot); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map6 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.qualifiers = new HashMap(2*_map6.size); - for (int _i7 = 0; _i7 < _map6.size; ++_i7) - { - String _key8; // required - TTypeQualifierValue _val9; // required - _key8 = iprot.readString(); - _val9 = new TTypeQualifierValue(); - _val9.read(iprot); - struct.qualifiers.put(_key8, _val9); - } - } - struct.setQualifiersIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java deleted file mode 100644 index 73dd45d3dd01a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java +++ /dev/null @@ -1,448 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TUnionTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUnionTypeEntry"); - - private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TUnionTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TUnionTypeEntryTupleSchemeFactory()); - } - - private Map nameToTypePtr; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME_TO_TYPE_PTR - return NAME_TO_TYPE_PTR; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUnionTypeEntry.class, metaDataMap); - } - - public TUnionTypeEntry() { - } - - public TUnionTypeEntry( - Map nameToTypePtr) - { - this(); - this.nameToTypePtr = nameToTypePtr; - } - - /** - * Performs a deep copy on other. - */ - public TUnionTypeEntry(TUnionTypeEntry other) { - if (other.isSetNameToTypePtr()) { - Map __this__nameToTypePtr = new HashMap(); - for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { - - String other_element_key = other_element.getKey(); - Integer other_element_value = other_element.getValue(); - - String __this__nameToTypePtr_copy_key = other_element_key; - - Integer __this__nameToTypePtr_copy_value = other_element_value; - - __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); - } - this.nameToTypePtr = __this__nameToTypePtr; - } - } - - public TUnionTypeEntry deepCopy() { - return new TUnionTypeEntry(this); - } - - @Override - public void clear() { - this.nameToTypePtr = null; - } - - public int getNameToTypePtrSize() { - return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); - } - - public void putToNameToTypePtr(String key, int val) { - if (this.nameToTypePtr == null) { - this.nameToTypePtr = new HashMap(); - } - this.nameToTypePtr.put(key, val); - } - - public Map getNameToTypePtr() { - return this.nameToTypePtr; - } - - public void setNameToTypePtr(Map nameToTypePtr) { - this.nameToTypePtr = nameToTypePtr; - } - - public void unsetNameToTypePtr() { - this.nameToTypePtr = null; - } - - /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ - public boolean isSetNameToTypePtr() { - return this.nameToTypePtr != null; - } - - public void setNameToTypePtrIsSet(boolean value) { - if (!value) { - this.nameToTypePtr = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME_TO_TYPE_PTR: - if (value == null) { - unsetNameToTypePtr(); - } else { - setNameToTypePtr((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME_TO_TYPE_PTR: - return getNameToTypePtr(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME_TO_TYPE_PTR: - return isSetNameToTypePtr(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TUnionTypeEntry) - return this.equals((TUnionTypeEntry)that); - return false; - } - - public boolean equals(TUnionTypeEntry that) { - if (that == null) - return false; - - boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); - boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); - if (this_present_nameToTypePtr || that_present_nameToTypePtr) { - if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) - return false; - if (!this.nameToTypePtr.equals(that.nameToTypePtr)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); - builder.append(present_nameToTypePtr); - if (present_nameToTypePtr) - builder.append(nameToTypePtr); - - return builder.toHashCode(); - } - - public int compareTo(TUnionTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TUnionTypeEntry typedOther = (TUnionTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetNameToTypePtr()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TUnionTypeEntry("); - boolean first = true; - - sb.append("nameToTypePtr:"); - if (this.nameToTypePtr == null) { - sb.append("null"); - } else { - sb.append(this.nameToTypePtr); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetNameToTypePtr()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TUnionTypeEntryStandardSchemeFactory implements SchemeFactory { - public TUnionTypeEntryStandardScheme getScheme() { - return new TUnionTypeEntryStandardScheme(); - } - } - - private static class TUnionTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // NAME_TO_TYPE_PTR - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map20 = iprot.readMapBegin(); - struct.nameToTypePtr = new HashMap(2*_map20.size); - for (int _i21 = 0; _i21 < _map20.size; ++_i21) - { - String _key22; // required - int _val23; // required - _key22 = iprot.readString(); - _val23 = iprot.readI32(); - struct.nameToTypePtr.put(_key22, _val23); - } - iprot.readMapEnd(); - } - struct.setNameToTypePtrIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.nameToTypePtr != null) { - oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); - for (Map.Entry _iter24 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter24.getKey()); - oprot.writeI32(_iter24.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TUnionTypeEntryTupleSchemeFactory implements SchemeFactory { - public TUnionTypeEntryTupleScheme getScheme() { - return new TUnionTypeEntryTupleScheme(); - } - } - - private static class TUnionTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - { - oprot.writeI32(struct.nameToTypePtr.size()); - for (Map.Entry _iter25 : struct.nameToTypePtr.entrySet()) - { - oprot.writeString(_iter25.getKey()); - oprot.writeI32(_iter25.getValue()); - } - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - { - org.apache.thrift.protocol.TMap _map26 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.nameToTypePtr = new HashMap(2*_map26.size); - for (int _i27 = 0; _i27 < _map26.size; ++_i27) - { - String _key28; // required - int _val29; // required - _key28 = iprot.readString(); - _val29 = iprot.readI32(); - struct.nameToTypePtr.put(_key28, _val29); - } - } - struct.setNameToTypePtrIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java deleted file mode 100644 index 3a111a2c8c2c6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java +++ /dev/null @@ -1,385 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package org.apache.hive.service.cli.thrift; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TUserDefinedTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUserDefinedTypeEntry"); - - private static final org.apache.thrift.protocol.TField TYPE_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("typeClassName", org.apache.thrift.protocol.TType.STRING, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new TUserDefinedTypeEntryStandardSchemeFactory()); - schemes.put(TupleScheme.class, new TUserDefinedTypeEntryTupleSchemeFactory()); - } - - private String typeClassName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - TYPE_CLASS_NAME((short)1, "typeClassName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPE_CLASS_NAME - return TYPE_CLASS_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPE_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("typeClassName", org.apache.thrift.TFieldRequirementType.REQUIRED, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUserDefinedTypeEntry.class, metaDataMap); - } - - public TUserDefinedTypeEntry() { - } - - public TUserDefinedTypeEntry( - String typeClassName) - { - this(); - this.typeClassName = typeClassName; - } - - /** - * Performs a deep copy on other. - */ - public TUserDefinedTypeEntry(TUserDefinedTypeEntry other) { - if (other.isSetTypeClassName()) { - this.typeClassName = other.typeClassName; - } - } - - public TUserDefinedTypeEntry deepCopy() { - return new TUserDefinedTypeEntry(this); - } - - @Override - public void clear() { - this.typeClassName = null; - } - - public String getTypeClassName() { - return this.typeClassName; - } - - public void setTypeClassName(String typeClassName) { - this.typeClassName = typeClassName; - } - - public void unsetTypeClassName() { - this.typeClassName = null; - } - - /** Returns true if field typeClassName is set (has been assigned a value) and false otherwise */ - public boolean isSetTypeClassName() { - return this.typeClassName != null; - } - - public void setTypeClassNameIsSet(boolean value) { - if (!value) { - this.typeClassName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPE_CLASS_NAME: - if (value == null) { - unsetTypeClassName(); - } else { - setTypeClassName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPE_CLASS_NAME: - return getTypeClassName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPE_CLASS_NAME: - return isSetTypeClassName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TUserDefinedTypeEntry) - return this.equals((TUserDefinedTypeEntry)that); - return false; - } - - public boolean equals(TUserDefinedTypeEntry that) { - if (that == null) - return false; - - boolean this_present_typeClassName = true && this.isSetTypeClassName(); - boolean that_present_typeClassName = true && that.isSetTypeClassName(); - if (this_present_typeClassName || that_present_typeClassName) { - if (!(this_present_typeClassName && that_present_typeClassName)) - return false; - if (!this.typeClassName.equals(that.typeClassName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_typeClassName = true && (isSetTypeClassName()); - builder.append(present_typeClassName); - if (present_typeClassName) - builder.append(typeClassName); - - return builder.toHashCode(); - } - - public int compareTo(TUserDefinedTypeEntry other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TUserDefinedTypeEntry typedOther = (TUserDefinedTypeEntry)other; - - lastComparison = Boolean.valueOf(isSetTypeClassName()).compareTo(typedOther.isSetTypeClassName()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetTypeClassName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeClassName, typedOther.typeClassName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TUserDefinedTypeEntry("); - boolean first = true; - - sb.append("typeClassName:"); - if (this.typeClassName == null) { - sb.append("null"); - } else { - sb.append(this.typeClassName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - if (!isSetTypeClassName()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeClassName' is unset! Struct:" + toString()); - } - - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class TUserDefinedTypeEntryStandardSchemeFactory implements SchemeFactory { - public TUserDefinedTypeEntryStandardScheme getScheme() { - return new TUserDefinedTypeEntryStandardScheme(); - } - } - - private static class TUserDefinedTypeEntryStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // TYPE_CLASS_NAME - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.typeClassName = iprot.readString(); - struct.setTypeClassNameIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.typeClassName != null) { - oprot.writeFieldBegin(TYPE_CLASS_NAME_FIELD_DESC); - oprot.writeString(struct.typeClassName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class TUserDefinedTypeEntryTupleSchemeFactory implements SchemeFactory { - public TUserDefinedTypeEntryTupleScheme getScheme() { - return new TUserDefinedTypeEntryTupleScheme(); - } - } - - private static class TUserDefinedTypeEntryTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - oprot.writeString(struct.typeClassName); - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - struct.typeClassName = iprot.readString(); - struct.setTypeClassNameIsSet(true); - } - } - -} - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java deleted file mode 100644 index 7e557aeccf5b0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java +++ /dev/null @@ -1,184 +0,0 @@ -/** - * 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.hive.service; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * AbstractService. - * - */ -public abstract class AbstractService implements Service { - - private static final Log LOG = LogFactory.getLog(AbstractService.class); - - /** - * Service state: initially {@link STATE#NOTINITED}. - */ - private Service.STATE state = STATE.NOTINITED; - - /** - * Service name. - */ - private final String name; - /** - * Service start time. Will be zero until the service is started. - */ - private long startTime; - - /** - * The configuration. Will be null until the service is initialized. - */ - private HiveConf hiveConf; - - /** - * List of state change listeners; it is final to ensure - * that it will never be null. - */ - private final List listeners = - new ArrayList(); - - /** - * Construct the service. - * - * @param name - * service name - */ - public AbstractService(String name) { - this.name = name; - } - - @Override - public synchronized Service.STATE getServiceState() { - return state; - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void init(HiveConf hiveConf) { - ensureCurrentState(STATE.NOTINITED); - this.hiveConf = hiveConf; - changeState(STATE.INITED); - LOG.info("Service:" + getName() + " is inited."); - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void start() { - startTime = System.currentTimeMillis(); - ensureCurrentState(STATE.INITED); - changeState(STATE.STARTED); - LOG.info("Service:" + getName() + " is started."); - } - - /** - * {@inheritDoc} - * - * @throws IllegalStateException - * if the current service state does not permit - * this action - */ - @Override - public synchronized void stop() { - if (state == STATE.STOPPED || - state == STATE.INITED || - state == STATE.NOTINITED) { - // already stopped, or else it was never - // started (eg another service failing canceled startup) - return; - } - ensureCurrentState(STATE.STARTED); - changeState(STATE.STOPPED); - LOG.info("Service:" + getName() + " is stopped."); - } - - @Override - public synchronized void register(ServiceStateChangeListener l) { - listeners.add(l); - } - - @Override - public synchronized void unregister(ServiceStateChangeListener l) { - listeners.remove(l); - } - - @Override - public String getName() { - return name; - } - - @Override - public synchronized HiveConf getHiveConf() { - return hiveConf; - } - - @Override - public long getStartTime() { - return startTime; - } - - /** - * Verify that a service is in a given state. - * - * @param currentState - * the desired state - * @throws IllegalStateException - * if the service state is different from - * the desired state - */ - private void ensureCurrentState(Service.STATE currentState) { - ServiceOperations.ensureCurrentState(state, currentState); - } - - /** - * Change to a new state and notify all listeners. - * This is a private method that is only invoked from synchronized methods, - * which avoid having to clone the listener list. It does imply that - * the state change listener methods should be short lived, as they - * will delay the state transition. - * - * @param newState - * new service state - */ - private void changeState(Service.STATE newState) { - state = newState; - // notify listeners - for (ServiceStateChangeListener l : listeners) { - l.stateChanged(this); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java deleted file mode 100644 index 897911872b80f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * 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.hive.service; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * CompositeService. - * - */ -public class CompositeService extends AbstractService { - - private static final Log LOG = LogFactory.getLog(CompositeService.class); - - private final List serviceList = new ArrayList(); - - public CompositeService(String name) { - super(name); - } - - public Collection getServices() { - return Collections.unmodifiableList(serviceList); - } - - protected synchronized void addService(Service service) { - serviceList.add(service); - } - - protected synchronized boolean removeService(Service service) { - return serviceList.remove(service); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - for (Service service : serviceList) { - service.init(hiveConf); - } - super.init(hiveConf); - } - - @Override - public synchronized void start() { - int i = 0; - try { - for (int n = serviceList.size(); i < n; i++) { - Service service = serviceList.get(i); - service.start(); - } - super.start(); - } catch (Throwable e) { - LOG.error("Error starting services " + getName(), e); - // Note that the state of the failed service is still INITED and not - // STARTED. Even though the last service is not started completely, still - // call stop() on all services including failed service to make sure cleanup - // happens. - stop(i); - throw new ServiceException("Failed to Start " + getName(), e); - } - - } - - @Override - public synchronized void stop() { - if (this.getServiceState() == STATE.STOPPED) { - // The base composite-service is already stopped, don't do anything again. - return; - } - if (serviceList.size() > 0) { - stop(serviceList.size() - 1); - } - super.stop(); - } - - private synchronized void stop(int numOfServicesStarted) { - // stop in reserve order of start - for (int i = numOfServicesStarted; i >= 0; i--) { - Service service = serviceList.get(i); - try { - service.stop(); - } catch (Throwable t) { - LOG.info("Error stopping " + service.getName(), t); - } - } - } - - /** - * JVM Shutdown hook for CompositeService which will stop the given - * CompositeService gracefully in case of JVM shutdown. - */ - public static class CompositeServiceShutdownHook implements Runnable { - - private final CompositeService compositeService; - - public CompositeServiceShutdownHook(CompositeService compositeService) { - this.compositeService = compositeService; - } - - @Override - public void run() { - try { - // Stop the Composite Service - compositeService.stop(); - } catch (Throwable t) { - LOG.info("Error stopping " + compositeService.getName(), t); - } - } - } - - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java deleted file mode 100644 index f2a80c9d5ffbc..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * 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.hive.service; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.logging.LogFactory; -import org.apache.commons.logging.Log; - -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; - -/** - * The cookie signer generates a signature based on SHA digest - * and appends it to the cookie value generated at the - * server side. It uses SHA digest algorithm to sign and verify signatures. - */ -public class CookieSigner { - private static final String SIGNATURE = "&s="; - private static final String SHA_STRING = "SHA"; - private byte[] secretBytes; - private static final Log LOG = LogFactory.getLog(CookieSigner.class); - - /** - * Constructor - * @param secret Secret Bytes - */ - public CookieSigner(byte[] secret) { - if (secret == null) { - throw new IllegalArgumentException(" NULL Secret Bytes"); - } - this.secretBytes = secret.clone(); - } - - /** - * Sign the cookie given the string token as input. - * @param str Input token - * @return Signed token that can be used to create a cookie - */ - public String signCookie(String str) { - if (str == null || str.isEmpty()) { - throw new IllegalArgumentException("NULL or empty string to sign"); - } - String signature = getSignature(str); - - if (LOG.isDebugEnabled()) { - LOG.debug("Signature generated for " + str + " is " + signature); - } - return str + SIGNATURE + signature; - } - - /** - * Verify a signed string and extracts the original string. - * @param signedStr The already signed string - * @return Raw Value of the string without the signature - */ - public String verifyAndExtract(String signedStr) { - int index = signedStr.lastIndexOf(SIGNATURE); - if (index == -1) { - throw new IllegalArgumentException("Invalid input sign: " + signedStr); - } - String originalSignature = signedStr.substring(index + SIGNATURE.length()); - String rawValue = signedStr.substring(0, index); - String currentSignature = getSignature(rawValue); - - if (LOG.isDebugEnabled()) { - LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); - } - if (!MessageDigest.isEqual(originalSignature.getBytes(), currentSignature.getBytes())) { - throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + - " current = " + currentSignature); - } - return rawValue; - } - - /** - * Get the signature of the input string based on SHA digest algorithm. - * @param str Input token - * @return Signed String - */ - private String getSignature(String str) { - try { - MessageDigest md = MessageDigest.getInstance(SHA_STRING); - md.update(str.getBytes()); - md.update(secretBytes); - byte[] digest = md.digest(); - return new Base64(0).encodeToString(digest); - } catch (NoSuchAlgorithmException ex) { - throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + - " " + ex.getMessage(), ex); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java deleted file mode 100644 index f16863c1b41aa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * 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.hive.service; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * ServiceOperations. - * - */ -public final class ServiceOperations { - private static final Log LOG = LogFactory.getLog(ServiceOperations.class); - - private ServiceOperations() { - } - - /** - * Verify that a service is in a given state. - * @param state the actual state a service is in - * @param expectedState the desired state - * @throws IllegalStateException if the service state is different from - * the desired state - */ - public static void ensureCurrentState(Service.STATE state, - Service.STATE expectedState) { - if (state != expectedState) { - throw new IllegalStateException("For this operation, the " + - "current service state must be " - + expectedState - + " instead of " + state); - } - } - - /** - * Initialize a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#NOTINITED} - * @param configuration the configuration to initialize the service with - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - - public static void init(Service service, HiveConf configuration) { - Service.STATE state = service.getServiceState(); - ensureCurrentState(state, Service.STATE.NOTINITED); - service.init(configuration); - } - - /** - * Start a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#INITED} - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - - public static void start(Service service) { - Service.STATE state = service.getServiceState(); - ensureCurrentState(state, Service.STATE.INITED); - service.start(); - } - - /** - * Initialize then start a service. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service that must be in the state - * {@link Service.STATE#NOTINITED} - * @param configuration the configuration to initialize the service with - * @throws RuntimeException on a state change failure - * @throws IllegalStateException if the service is in the wrong state - */ - public static void deploy(Service service, HiveConf configuration) { - init(service, configuration); - start(service); - } - - /** - * Stop a service. - * - * Do nothing if the service is null or not in a state in which it can be/needs to be stopped. - * - * The service state is checked before the operation begins. - * This process is not thread safe. - * @param service a service or null - */ - public static void stop(Service service) { - if (service != null) { - Service.STATE state = service.getServiceState(); - if (state == Service.STATE.STARTED) { - service.stop(); - } - } - } - - /** - * Stop a service; if it is null do nothing. Exceptions are caught and - * logged at warn level. (but not Throwables). This operation is intended to - * be used in cleanup operations - * - * @param service a service; may be null - * @return any exception that was caught; null if none was. - */ - public static Exception stopQuietly(Service service) { - try { - stop(service); - } catch (Exception e) { - LOG.warn("When stopping the service " + service.getName() - + " : " + e, - e); - return e; - } - return null; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java deleted file mode 100644 index edb5eff9615bf..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * 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.hive.service; - -public class ServiceUtils { - - /* - * Get the index separating the user name from domain name (the user's name up - * to the first '/' or '@'). - * - * @param userName full user name. - * @return index of domain match or -1 if not found - */ - public static int indexOfDomainMatch(String userName) { - if (userName == null) { - return -1; - } - - int idx = userName.indexOf('/'); - int idx2 = userName.indexOf('@'); - int endIdx = Math.min(idx, idx2); // Use the earlier match. - // Unless at least one of '/' or '@' was not found, in - // which case, user the latter match. - if (endIdx == -1) { - endIdx = Math.max(idx, idx2); - } - return endIdx; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java deleted file mode 100644 index 10000f12ab329..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java +++ /dev/null @@ -1,419 +0,0 @@ -/** - * 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.hive.service.auth; - -import java.io.IOException; -import java.lang.reflect.Field; -import java.lang.reflect.Method; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; - -import javax.net.ssl.SSLServerSocket; -import javax.security.auth.login.LoginException; -import javax.security.sasl.Sasl; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.DBTokenStore; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.authorize.ProxyUsers; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSSLTransportFactory; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportException; -import org.apache.thrift.transport.TTransportFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class helps in some aspects of authentication. It creates the proper Thrift classes for the - * given configuration as well as helps with authenticating requests. - */ -public class HiveAuthFactory { - private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class); - - - public enum AuthTypes { - NOSASL("NOSASL"), - NONE("NONE"), - LDAP("LDAP"), - KERBEROS("KERBEROS"), - CUSTOM("CUSTOM"), - PAM("PAM"); - - private final String authType; - - AuthTypes(String authType) { - this.authType = authType; - } - - public String getAuthName() { - return authType; - } - - } - - private HadoopThriftAuthBridge.Server saslServer; - private String authTypeStr; - private final String transportMode; - private final HiveConf conf; - - public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; - public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; - - private static Field keytabFile = null; - private static Method getKeytab = null; - static { - Class clz = UserGroupInformation.class; - try { - keytabFile = clz.getDeclaredField("keytabFile"); - keytabFile.setAccessible(true); - } catch (NoSuchFieldException nfe) { - LOG.debug("Cannot find private field \"keytabFile\" in class: " + - UserGroupInformation.class.getCanonicalName(), nfe); - keytabFile = null; - } - - try { - getKeytab = clz.getDeclaredMethod("getKeytab"); - getKeytab.setAccessible(true); - } catch(NoSuchMethodException nme) { - LOG.debug("Cannot find private method \"getKeytab\" in class:" + - UserGroupInformation.class.getCanonicalName(), nme); - getKeytab = null; - } - } - - public HiveAuthFactory(HiveConf conf) throws TTransportException, IOException { - this.conf = conf; - transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); - authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION); - - // In http mode we use NOSASL as the default auth type - if ("http".equalsIgnoreCase(transportMode)) { - if (authTypeStr == null) { - authTypeStr = AuthTypes.NOSASL.getAuthName(); - } - } else { - if (authTypeStr == null) { - authTypeStr = AuthTypes.NONE.getAuthName(); - } - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); - String keytab = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); - if (needUgiLogin(UserGroupInformation.getCurrentUser(), - SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keytab)) { - saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(principal, keytab); - } else { - // Using the default constructor to avoid unnecessary UGI login. - saslServer = new HadoopThriftAuthBridge.Server(); - } - - // start delegation token manager - try { - // rawStore is only necessary for DBTokenStore - Object rawStore = null; - String tokenStoreClass = conf.getVar(HiveConf.ConfVars.METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS); - - if (tokenStoreClass.equals(DBTokenStore.class.getName())) { - HMSHandler baseHandler = new HiveMetaStore.HMSHandler( - "new db based metaserver", conf, true); - rawStore = baseHandler.getMS(); - } - - saslServer.startDelegationTokenSecretManager(conf, rawStore, ServerMode.HIVESERVER2); - } - catch (MetaException|IOException e) { - throw new TTransportException("Failed to start token manager", e); - } - } - } - } - - public Map getSaslProperties() { - Map saslProps = new HashMap(); - SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP)); - saslProps.put(Sasl.QOP, saslQOP.toString()); - saslProps.put(Sasl.SERVER_AUTH, "true"); - return saslProps; - } - - public TTransportFactory getAuthTransFactory() throws LoginException { - TTransportFactory transportFactory; - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - try { - transportFactory = saslServer.createTransportFactory(getSaslProperties()); - } catch (TTransportException e) { - throw new LoginException(e.getMessage()); - } - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) { - transportFactory = new TTransportFactory(); - } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) { - transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); - } else { - throw new LoginException("Unsupported authentication type " + authTypeStr); - } - return transportFactory; - } - - /** - * Returns the thrift processor factory for HiveServer2 running in binary mode - * @param service - * @return - * @throws LoginException - */ - public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException { - if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service); - } else { - return PlainSaslHelper.getPlainProcessorFactory(service); - } - } - - public String getRemoteUser() { - return saslServer == null ? null : saslServer.getRemoteUser(); - } - - public String getIpAddress() { - if (saslServer == null || saslServer.getRemoteAddress() == null) { - return null; - } else { - return saslServer.getRemoteAddress().getHostAddress(); - } - } - - // Perform kerberos login using the hadoop shim API if the configuration is available - public static void loginFromKeytab(HiveConf hiveConf) throws IOException { - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured"); - } else { - UserGroupInformation.loginUserFromKeytab(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); - } - } - - // Perform SPNEGO login using the hadoop shim API if the configuration is available - public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf) - throws IOException { - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured"); - } else { - return UserGroupInformation.loginUserFromKeytabAndReturnUGI(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); - } - } - - public static TTransport getSocketTransport(String host, int port, int loginTimeout) { - return new TSocket(host, port, loginTimeout); - } - - public static TTransport getSSLSocket(String host, int port, int loginTimeout) - throws TTransportException { - return TSSLTransportFactory.getClientSocket(host, port, loginTimeout); - } - - public static TTransport getSSLSocket(String host, int port, int loginTimeout, - String trustStorePath, String trustStorePassWord) throws TTransportException { - TSSLTransportFactory.TSSLTransportParameters params = - new TSSLTransportFactory.TSSLTransportParameters(); - params.setTrustStore(trustStorePath, trustStorePassWord); - params.requireClientAuth(true); - return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params); - } - - public static TServerSocket getServerSocket(String hiveHost, int portNum) - throws TTransportException { - InetSocketAddress serverAddress; - if (hiveHost == null || hiveHost.isEmpty()) { - // Wildcard bind - serverAddress = new InetSocketAddress(portNum); - } else { - serverAddress = new InetSocketAddress(hiveHost, portNum); - } - return new TServerSocket(serverAddress); - } - - public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, - String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, - UnknownHostException { - TSSLTransportFactory.TSSLTransportParameters params = - new TSSLTransportFactory.TSSLTransportParameters(); - params.setKeyStore(keyStorePath, keyStorePassWord); - InetSocketAddress serverAddress; - if (hiveHost == null || hiveHost.isEmpty()) { - // Wildcard bind - serverAddress = new InetSocketAddress(portNum); - } else { - serverAddress = new InetSocketAddress(hiveHost, portNum); - } - TServerSocket thriftServerSocket = - TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params); - if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) { - List sslVersionBlacklistLocal = new ArrayList(); - for (String sslVersion : sslVersionBlacklist) { - sslVersionBlacklistLocal.add(sslVersion.trim().toLowerCase(Locale.ROOT)); - } - SSLServerSocket sslServerSocket = (SSLServerSocket) thriftServerSocket.getServerSocket(); - List enabledProtocols = new ArrayList(); - for (String protocol : sslServerSocket.getEnabledProtocols()) { - if (sslVersionBlacklistLocal.contains(protocol.toLowerCase(Locale.ROOT))) { - LOG.debug("Disabling SSL Protocol: " + protocol); - } else { - enabledProtocols.add(protocol); - } - } - sslServerSocket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); - LOG.info("SSL Server Socket Enabled Protocols: " - + Arrays.toString(sslServerSocket.getEnabledProtocols())); - } - return thriftServerSocket; - } - - // retrieve delegation token for the given user - public String getDelegationToken(String owner, String renewer) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - - try { - String tokenStr = saslServer.getDelegationTokenWithService(owner, renewer, HS2_CLIENT_TOKEN); - if (tokenStr == null || tokenStr.isEmpty()) { - throw new HiveSQLException( - "Received empty retrieving delegation token for user " + owner, "08S01"); - } - return tokenStr; - } catch (IOException e) { - throw new HiveSQLException( - "Error retrieving delegation token for user " + owner, "08S01", e); - } catch (InterruptedException e) { - throw new HiveSQLException("delegation token retrieval interrupted", "08S01", e); - } - } - - // cancel given delegation token - public void cancelDelegationToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - saslServer.cancelDelegationToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error canceling delegation token " + delegationToken, "08S01", e); - } - } - - public void renewDelegationToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - saslServer.renewDelegationToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error renewing delegation token " + delegationToken, "08S01", e); - } - } - - public String getUserFromToken(String delegationToken) throws HiveSQLException { - if (saslServer == null) { - throw new HiveSQLException( - "Delegation token only supported over kerberos authentication", "08S01"); - } - try { - return saslServer.getUserFromToken(delegationToken); - } catch (IOException e) { - throw new HiveSQLException( - "Error extracting user from delegation token " + delegationToken, "08S01", e); - } - } - - public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress, - HiveConf hiveConf) throws HiveSQLException { - try { - UserGroupInformation sessionUgi; - if (UserGroupInformation.isSecurityEnabled()) { - KerberosNameShim kerbName = ShimLoader.getHadoopShims().getKerberosNameShim(realUser); - sessionUgi = UserGroupInformation.createProxyUser( - kerbName.getServiceName(), UserGroupInformation.getLoginUser()); - } else { - sessionUgi = UserGroupInformation.createRemoteUser(realUser); - } - if (!proxyUser.equalsIgnoreCase(realUser)) { - ProxyUsers.refreshSuperUserGroupsConfiguration(hiveConf); - ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), - ipAddress, hiveConf); - } - } catch (IOException e) { - throw new HiveSQLException( - "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, "08S01", e); - } - } - - public static boolean needUgiLogin(UserGroupInformation ugi, String principal, String keytab) { - return null == ugi || !ugi.hasKerberosCredentials() || !ugi.getUserName().equals(principal) || - !Objects.equals(keytab, getKeytabFromUgi()); - } - - private static String getKeytabFromUgi() { - synchronized (UserGroupInformation.class) { - try { - if (keytabFile != null) { - return (String) keytabFile.get(null); - } else if (getKeytab != null) { - return (String) getKeytab.invoke(UserGroupInformation.getCurrentUser()); - } else { - return null; - } - } catch (Exception e) { - LOG.debug("Fail to get keytabFile path via reflection", e); - return null; - } - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java deleted file mode 100644 index f7375ee707830..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ /dev/null @@ -1,189 +0,0 @@ -/** - * 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.hive.service.auth; - -import java.security.AccessControlContext; -import java.security.AccessController; -import java.security.PrivilegedExceptionAction; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.StringTokenizer; - -import javax.security.auth.Subject; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.http.protocol.BasicHttpContext; -import org.apache.http.protocol.HttpContext; -import org.ietf.jgss.GSSContext; -import org.ietf.jgss.GSSManager; -import org.ietf.jgss.GSSName; -import org.ietf.jgss.Oid; - -/** - * Utility functions for HTTP mode authentication. - */ -public final class HttpAuthUtils { - public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; - public static final String AUTHORIZATION = "Authorization"; - public static final String BASIC = "Basic"; - public static final String NEGOTIATE = "Negotiate"; - private static final Log LOG = LogFactory.getLog(HttpAuthUtils.class); - private static final String COOKIE_ATTR_SEPARATOR = "&"; - private static final String COOKIE_CLIENT_USER_NAME = "cu"; - private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; - private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; - private static final Set COOKIE_ATTRIBUTES = - new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); - - /** - * @return Stringified Base64 encoded kerberosAuthHeader on success - * @throws Exception - */ - public static String getKerberosServiceTicket(String principal, String host, - String serverHttpUrl, boolean assumeSubject) throws Exception { - String serverPrincipal = - ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); - if (assumeSubject) { - // With this option, we're assuming that the external application, - // using the JDBC driver has done a JAAS kerberos login already - AccessControlContext context = AccessController.getContext(); - Subject subject = Subject.getSubject(context); - if (subject == null) { - throw new Exception("The Subject is not set"); - } - return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); - } else { - // JAAS login from ticket cache to setup the client UserGroupInformation - UserGroupInformation clientUGI = - ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); - return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); - } - } - - /** - * Creates and returns a HS2 cookie token. - * @param clientUserName Client User name. - * @return An unsigned cookie token generated from input parameters. - * The final cookie generated is of the following format : - * {@code cu=&rn=&s=} - */ - public static String createCookieToken(String clientUserName) { - StringBuffer sb = new StringBuffer(); - sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) - .append(COOKIE_ATTR_SEPARATOR); - sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) - .append((new Random(System.currentTimeMillis())).nextLong()); - return sb.toString(); - } - - /** - * Parses a cookie token to retrieve client user name. - * @param tokenStr Token String. - * @return A valid user name if input is of valid format, else returns null. - */ - public static String getUserNameFromCookieToken(String tokenStr) { - Map map = splitCookieToken(tokenStr); - - if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { - LOG.error("Invalid token with missing attributes " + tokenStr); - return null; - } - return map.get(COOKIE_CLIENT_USER_NAME); - } - - /** - * Splits the cookie token into attributes pairs. - * @param str input token. - * @return a map with the attribute pairs of the token if the input is valid. - * Else, returns null. - */ - private static Map splitCookieToken(String tokenStr) { - Map map = new HashMap(); - StringTokenizer st = new StringTokenizer(tokenStr, COOKIE_ATTR_SEPARATOR); - - while (st.hasMoreTokens()) { - String part = st.nextToken(); - int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); - if (separator == -1) { - LOG.error("Invalid token string " + tokenStr); - return null; - } - String key = part.substring(0, separator); - String value = part.substring(separator + 1); - map.put(key, value); - } - return map; - } - - - private HttpAuthUtils() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - /** - * We'll create an instance of this class within a doAs block so that the client's TGT credentials - * can be read from the Subject - */ - public static class HttpKerberosClientAction implements PrivilegedExceptionAction { - public static final String HTTP_RESPONSE = "HTTP_RESPONSE"; - public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; - private final String serverPrincipal; - private final String serverHttpUrl; - private final Base64 base64codec; - private final HttpContext httpContext; - - public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { - this.serverPrincipal = serverPrincipal; - this.serverHttpUrl = serverHttpUrl; - base64codec = new Base64(0); - httpContext = new BasicHttpContext(); - httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); - } - - @Override - public String run() throws Exception { - // This Oid for Kerberos GSS-API mechanism. - Oid mechOid = new Oid("1.2.840.113554.1.2.2"); - // Oid for kerberos principal name - Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); - GSSManager manager = GSSManager.getInstance(); - // GSS name for server - GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); - // Create a GSSContext for authentication with the service. - // We're passing client credentials as null since we want them to be read from the Subject. - GSSContext gssContext = - manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME); - gssContext.requestMutualAuth(false); - // Establish context - byte[] inToken = new byte[0]; - byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); - gssContext.dispose(); - // Base64 encoded and stringified token for server - return new String(base64codec.encode(outToken)); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java deleted file mode 100644 index 52eb752f1e026..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ /dev/null @@ -1,111 +0,0 @@ -/** - * 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.hive.service.auth; - -import java.io.IOException; -import java.util.Map; -import javax.security.sasl.SaslException; - -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; -import org.apache.hive.service.cli.thrift.TCLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessor; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TTransport; - -public final class KerberosSaslHelper { - - public static TProcessorFactory getKerberosProcessorFactory(Server saslServer, - ThriftCLIService service) { - return new CLIServiceProcessorFactory(saslServer, service); - } - - public static TTransport getKerberosTransport(String principal, String host, - TTransport underlyingTransport, Map saslProps, boolean assumeSubject) - throws SaslException { - try { - String[] names = principal.split("[/@]"); - if (names.length != 3) { - throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal); - } - - if (assumeSubject) { - return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); - } else { - HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); - return authBridge.createClientTransport(principal, host, "KERBEROS", null, - underlyingTransport, saslProps); - } - } catch (IOException e) { - throw new SaslException("Failed to open client transport", e); - } - } - - public static TTransport createSubjectAssumedTransport(String principal, - TTransport underlyingTransport, Map saslProps) throws IOException { - String[] names = principal.split("[/@]"); - try { - TTransport saslTransport = - new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, - underlyingTransport); - return new TSubjectAssumingTransport(saslTransport); - } catch (SaslException se) { - throw new IOException("Could not instantiate SASL transport", se); - } - } - - public static TTransport getTokenTransport(String tokenStr, String host, - TTransport underlyingTransport, Map saslProps) throws SaslException { - HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); - - try { - return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, - saslProps); - } catch (IOException e) { - throw new SaslException("Failed to open client transport", e); - } - } - - private KerberosSaslHelper() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - private static class CLIServiceProcessorFactory extends TProcessorFactory { - - private final ThriftCLIService service; - private final Server saslServer; - - CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { - super(null); - this.service = service; - this.saslServer = saslServer; - } - - @Override - public TProcessor getProcessor(TTransport trans) { - TProcessor sqlProcessor = new TCLIService.Processor(service); - return saslServer.wrapNonAssumingProcessor(sqlProcessor); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java deleted file mode 100644 index afc144199f1e8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java +++ /dev/null @@ -1,154 +0,0 @@ -/** - * 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.hive.service.auth; - -import java.io.IOException; -import java.security.Security; -import java.util.HashMap; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.auth.login.LoginException; -import javax.security.sasl.AuthenticationException; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.SaslException; - -import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; -import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.thrift.TProcessor; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TSaslServerTransport; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportFactory; - -public final class PlainSaslHelper { - - public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) { - return new SQLPlainProcessorFactory(service); - } - - // Register Plain SASL server provider - static { - Security.addProvider(new SaslPlainProvider()); - } - - public static TTransportFactory getPlainTransportFactory(String authTypeStr) - throws LoginException { - TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory(); - try { - saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap(), - new PlainServerCallbackHandler(authTypeStr)); - } catch (AuthenticationException e) { - throw new LoginException("Error setting callback handler" + e); - } - return saslFactory; - } - - public static TTransport getPlainTransport(String username, String password, - TTransport underlyingTransport) throws SaslException { - return new TSaslClientTransport("PLAIN", null, null, null, new HashMap(), - new PlainCallbackHandler(username, password), underlyingTransport); - } - - private PlainSaslHelper() { - throw new UnsupportedOperationException("Can't initialize class"); - } - - private static final class PlainServerCallbackHandler implements CallbackHandler { - - private final AuthMethods authMethod; - - PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException { - authMethod = AuthMethods.getValidAuthMethod(authMethodStr); - } - - @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { - String username = null; - String password = null; - AuthorizeCallback ac = null; - - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - NameCallback nc = (NameCallback) callback; - username = nc.getName(); - } else if (callback instanceof PasswordCallback) { - PasswordCallback pc = (PasswordCallback) callback; - password = new String(pc.getPassword()); - } else if (callback instanceof AuthorizeCallback) { - ac = (AuthorizeCallback) callback; - } else { - throw new UnsupportedCallbackException(callback); - } - } - PasswdAuthenticationProvider provider = - AuthenticationProviderFactory.getAuthenticationProvider(authMethod); - provider.Authenticate(username, password); - if (ac != null) { - ac.setAuthorized(true); - } - } - } - - public static class PlainCallbackHandler implements CallbackHandler { - - private final String username; - private final String password; - - public PlainCallbackHandler(String username, String password) { - this.username = username; - this.password = password; - } - - @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - NameCallback nameCallback = (NameCallback) callback; - nameCallback.setName(username); - } else if (callback instanceof PasswordCallback) { - PasswordCallback passCallback = (PasswordCallback) callback; - passCallback.setPassword(password.toCharArray()); - } else { - throw new UnsupportedCallbackException(callback); - } - } - } - } - - private static final class SQLPlainProcessorFactory extends TProcessorFactory { - - private final ThriftCLIService service; - - SQLPlainProcessorFactory(ThriftCLIService service) { - super(null); - this.service = service; - } - - @Override - public TProcessor getProcessor(TTransport trans) { - return new TSetIpAddressProcessor(service); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java deleted file mode 100644 index 9a61ad49942c8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * 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.hive.service.auth; - -import org.apache.hive.service.cli.thrift.TCLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TProtocol; -import org.apache.thrift.transport.TSaslClientTransport; -import org.apache.thrift.transport.TSaslServerTransport; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class is responsible for setting the ipAddress for operations executed via HiveServer2. - * - * - IP address is only set for operations that calls listeners with hookContext - * - IP address is only set if the underlying transport mechanism is socket - * - * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext - */ -public class TSetIpAddressProcessor extends TCLIService.Processor { - - private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName()); - - public TSetIpAddressProcessor(Iface iface) { - super(iface); - } - - @Override - public boolean process(final TProtocol in, final TProtocol out) throws TException { - setIpAddress(in); - setUserName(in); - try { - return super.process(in, out); - } finally { - THREAD_LOCAL_USER_NAME.remove(); - THREAD_LOCAL_IP_ADDRESS.remove(); - } - } - - private void setUserName(final TProtocol in) { - TTransport transport = in.getTransport(); - if (transport instanceof TSaslServerTransport) { - String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID(); - THREAD_LOCAL_USER_NAME.set(userName); - } - } - - protected void setIpAddress(final TProtocol in) { - TTransport transport = in.getTransport(); - TSocket tSocket = getUnderlyingSocketFromTransport(transport); - if (tSocket == null) { - LOGGER.warn("Unknown Transport, cannot determine ipAddress"); - } else { - THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().getHostAddress()); - } - } - - private TSocket getUnderlyingSocketFromTransport(TTransport transport) { - while (transport != null) { - if (transport instanceof TSaslServerTransport) { - transport = ((TSaslServerTransport) transport).getUnderlyingTransport(); - } - if (transport instanceof TSaslClientTransport) { - transport = ((TSaslClientTransport) transport).getUnderlyingTransport(); - } - if (transport instanceof TSocket) { - return (TSocket) transport; - } - } - return null; - } - - private static final ThreadLocal THREAD_LOCAL_IP_ADDRESS = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - private static final ThreadLocal THREAD_LOCAL_USER_NAME = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static String getUserIpAddress() { - return THREAD_LOCAL_IP_ADDRESS.get(); - } - - public static String getUserName() { - return THREAD_LOCAL_USER_NAME.get(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java deleted file mode 100644 index 791ddcbd2c5b6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java +++ /dev/null @@ -1,507 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import javax.security.auth.login.LoginException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.operation.Operation; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.HiveServer2; - -/** - * CLIService. - * - */ -public class CLIService extends CompositeService implements ICLIService { - - public static final TProtocolVersion SERVER_VERSION; - - static { - TProtocolVersion[] protocols = TProtocolVersion.values(); - SERVER_VERSION = protocols[protocols.length - 1]; - } - - private final Log LOG = LogFactory.getLog(CLIService.class.getName()); - - private HiveConf hiveConf; - private SessionManager sessionManager; - private UserGroupInformation serviceUGI; - private UserGroupInformation httpUGI; - // The HiveServer2 instance running this service - private final HiveServer2 hiveServer2; - - public CLIService(HiveServer2 hiveServer2) { - super(CLIService.class.getSimpleName()); - this.hiveServer2 = hiveServer2; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - sessionManager = new SessionManager(hiveServer2); - addService(sessionManager); - // If the hadoop cluster is secure, do a kerberos login for the service from the keytab - if (UserGroupInformation.isSecurityEnabled()) { - try { - HiveAuthFactory.loginFromKeytab(hiveConf); - this.serviceUGI = Utils.getUGI(); - } catch (IOException e) { - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); - } catch (LoginException e) { - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); - } - - // Also try creating a UGI object for the SPNego principal - String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); - if (principal.isEmpty() || keyTabFile.isEmpty()) { - LOG.info("SPNego httpUGI not created, spNegoPrincipal: " + principal + - ", ketabFile: " + keyTabFile); - } else { - try { - this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); - LOG.info("SPNego httpUGI successfully created."); - } catch (IOException e) { - LOG.warn("SPNego httpUGI creation failed: ", e); - } - } - } - // creates connection to HMS and thus *must* occur after kerberos login above - try { - applyAuthorizationConfigPolicy(hiveConf); - } catch (Exception e) { - throw new RuntimeException("Error applying authorization policy on hive configuration: " - + e.getMessage(), e); - } - setupBlockedUdfs(); - super.init(hiveConf); - } - - private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException, - MetaException { - // authorization setup using SessionState should be revisited eventually, as - // authorization and authentication are not session specific settings - SessionState ss = new SessionState(newHiveConf); - ss.setIsHiveServerQuery(true); - SessionState.start(ss); - ss.applyAuthorizationPolicy(); - } - - private void setupBlockedUdfs() { - FunctionRegistry.setupPermissionsForBuiltinUDFs( - hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST), - hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST)); - } - - public UserGroupInformation getServiceUGI() { - return this.serviceUGI; - } - - public UserGroupInformation getHttpUGI() { - return this.httpUGI; - } - - @Override - public synchronized void start() { - super.start(); - // Initialize and test a connection to the metastore - IMetaStoreClient metastoreClient = null; - try { - metastoreClient = new HiveMetaStoreClient(hiveConf); - metastoreClient.getDatabases("default"); - } catch (Exception e) { - throw new ServiceException("Unable to connect to MetaStore!", e); - } - finally { - if (metastoreClient != null) { - metastoreClient.close(); - } - } - } - - @Override - public synchronized void stop() { - super.stop(); - } - - /** - * @deprecated Use {@link #openSession(TProtocolVersion, String, String, String, Map)} - */ - @Deprecated - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, - Map configuration) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /** - * @deprecated Use {@link #openSessionWithImpersonation(TProtocolVersion, String, String, String, Map, String)} - */ - @Deprecated - public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, - String password, Map configuration, String delegationToken) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSessionWithImpersonation()"); - return sessionHandle; - } - - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map configuration) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, - String password, String ipAddress, Map configuration, String delegationToken) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, Map configuration) - throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, false, null); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, Map configuration, - String delegationToken) throws HiveSQLException { - SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, - true, delegationToken); - LOG.debug(sessionHandle + ": openSession()"); - return sessionHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) - throws HiveSQLException { - sessionManager.closeSession(sessionHandle); - LOG.debug(sessionHandle + ": closeSession()"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) - throws HiveSQLException { - GetInfoValue infoValue = sessionManager.getSession(sessionHandle) - .getInfo(getInfoType); - LOG.debug(sessionHandle + ": getInfo()"); - return infoValue; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .executeStatement(statement, confOverlay); - LOG.debug(sessionHandle + ": executeStatement()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .executeStatementAsync(statement, confOverlay); - LOG.debug(sessionHandle + ": executeStatementAsync()"); - return opHandle; - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTypeInfo(); - LOG.debug(sessionHandle + ": getTypeInfo()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getCatalogs(); - LOG.debug(sessionHandle + ": getCatalogs()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, - String catalogName, String schemaName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getSchemas(catalogName, schemaName); - LOG.debug(sessionHandle + ": getSchemas()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, List tableTypes) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTables(catalogName, schemaName, tableName, tableTypes); - LOG.debug(sessionHandle + ": getTables()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getTableTypes(); - LOG.debug(sessionHandle + ": getTableTypes()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getColumns(catalogName, schemaName, tableName, columnName); - LOG.debug(sessionHandle + ": getColumns()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException { - OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getFunctions(catalogName, schemaName, functionName); - LOG.debug(sessionHandle + ": getFunctions()"); - return opHandle; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException { - Operation operation = sessionManager.getOperationManager().getOperation(opHandle); - /** - * If this is a background operation run asynchronously, - * we block for a configured duration, before we return - * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). - * However, if the background operation is complete, we return immediately. - */ - if (operation.shouldRunAsync()) { - HiveConf conf = operation.getParentSession().getHiveConf(); - long timeout = HiveConf.getTimeVar(conf, - HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); - try { - operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - // No Op, return to the caller since long polling timeout has expired - LOG.trace(opHandle + ": Long polling timed out"); - } catch (CancellationException e) { - // The background operation thread was cancelled - LOG.trace(opHandle + ": The background operation was cancelled", e); - } catch (ExecutionException e) { - // The background operation thread was aborted - LOG.warn(opHandle + ": The background operation was aborted", e); - } catch (InterruptedException e) { - // No op, this thread was interrupted - // In this case, the call might return sooner than long polling timeout - } - } - OperationStatus opStatus = operation.getStatus(); - LOG.debug(opHandle + ": getOperationStatus()"); - return opStatus; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) - throws HiveSQLException { - sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().cancelOperation(opHandle); - LOG.debug(opHandle + ": cancelOperation()"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) - throws HiveSQLException { - sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().closeOperation(opHandle); - LOG.debug(opHandle + ": closeOperation"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException { - TableSchema tableSchema = sessionManager.getOperationManager() - .getOperation(opHandle).getParentSession().getResultSetMetadata(opHandle); - LOG.debug(opHandle + ": getResultSetMetadata()"); - return tableSchema; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public RowSet fetchResults(OperationHandle opHandle) - throws HiveSQLException { - return fetchResults(opHandle, Operation.DEFAULT_FETCH_ORIENTATION, - Operation.DEFAULT_FETCH_MAX_ROWS, FetchType.QUERY_OUTPUT); - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle) - .getParentSession().fetchResults(opHandle, orientation, maxRows, fetchType); - LOG.debug(opHandle + ": fetchResults()"); - return rowSet; - } - - // obtain delegation token for the give user from metastore - public synchronized String getDelegationTokenFromMetaStore(String owner) - throws HiveSQLException, UnsupportedOperationException, LoginException, IOException { - if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) || - !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { - throw new UnsupportedOperationException( - "delegation token is can only be obtained for a secure remote metastore"); - } - - try { - Hive.closeCurrent(); - return Hive.get(hiveConf).getDelegationToken(owner, owner); - } catch (HiveException e) { - if (e.getCause() instanceof UnsupportedOperationException) { - throw (UnsupportedOperationException)e.getCause(); - } else { - throw new HiveSQLException("Error connect metastore to setup impersonation", e); - } - } - } - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - String delegationToken = sessionManager.getSession(sessionHandle) - .getDelegationToken(authFactory, owner, renewer); - LOG.info(sessionHandle + ": getDelegationToken()"); - return delegationToken; - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); - LOG.info(sessionHandle + ": cancelDelegationToken()"); - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); - LOG.info(sessionHandle + ": renewDelegationToken()"); - } - - public SessionManager getSessionManager() { - return sessionManager; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java deleted file mode 100644 index 26d0f718f383a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java +++ /dev/null @@ -1,423 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.nio.ByteBuffer; -import java.util.AbstractList; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.List; - -import com.google.common.primitives.Booleans; -import com.google.common.primitives.Bytes; -import com.google.common.primitives.Doubles; -import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import com.google.common.primitives.Shorts; -import org.apache.hive.service.cli.thrift.TBinaryColumn; -import org.apache.hive.service.cli.thrift.TBoolColumn; -import org.apache.hive.service.cli.thrift.TByteColumn; -import org.apache.hive.service.cli.thrift.TColumn; -import org.apache.hive.service.cli.thrift.TDoubleColumn; -import org.apache.hive.service.cli.thrift.TI16Column; -import org.apache.hive.service.cli.thrift.TI32Column; -import org.apache.hive.service.cli.thrift.TI64Column; -import org.apache.hive.service.cli.thrift.TStringColumn; - -/** - * Column. - */ -public class Column extends AbstractList { - - private static final int DEFAULT_SIZE = 100; - - private final Type type; - - private BitSet nulls; - - private int size; - private boolean[] boolVars; - private byte[] byteVars; - private short[] shortVars; - private int[] intVars; - private long[] longVars; - private double[] doubleVars; - private List stringVars; - private List binaryVars; - - public Column(Type type, BitSet nulls, Object values) { - this.type = type; - this.nulls = nulls; - if (type == Type.BOOLEAN_TYPE) { - boolVars = (boolean[]) values; - size = boolVars.length; - } else if (type == Type.TINYINT_TYPE) { - byteVars = (byte[]) values; - size = byteVars.length; - } else if (type == Type.SMALLINT_TYPE) { - shortVars = (short[]) values; - size = shortVars.length; - } else if (type == Type.INT_TYPE) { - intVars = (int[]) values; - size = intVars.length; - } else if (type == Type.BIGINT_TYPE) { - longVars = (long[]) values; - size = longVars.length; - } else if (type == Type.DOUBLE_TYPE) { - doubleVars = (double[]) values; - size = doubleVars.length; - } else if (type == Type.BINARY_TYPE) { - binaryVars = (List) values; - size = binaryVars.size(); - } else if (type == Type.STRING_TYPE) { - stringVars = (List) values; - size = stringVars.size(); - } else { - throw new IllegalStateException("invalid union object"); - } - } - - public Column(Type type) { - nulls = new BitSet(); - switch (type) { - case BOOLEAN_TYPE: - boolVars = new boolean[DEFAULT_SIZE]; - break; - case TINYINT_TYPE: - byteVars = new byte[DEFAULT_SIZE]; - break; - case SMALLINT_TYPE: - shortVars = new short[DEFAULT_SIZE]; - break; - case INT_TYPE: - intVars = new int[DEFAULT_SIZE]; - break; - case BIGINT_TYPE: - longVars = new long[DEFAULT_SIZE]; - break; - case FLOAT_TYPE: - case DOUBLE_TYPE: - type = Type.DOUBLE_TYPE; - doubleVars = new double[DEFAULT_SIZE]; - break; - case BINARY_TYPE: - binaryVars = new ArrayList(); - break; - default: - type = Type.STRING_TYPE; - stringVars = new ArrayList(); - } - this.type = type; - } - - public Column(TColumn colValues) { - if (colValues.isSetBoolVal()) { - type = Type.BOOLEAN_TYPE; - nulls = toBitset(colValues.getBoolVal().getNulls()); - boolVars = Booleans.toArray(colValues.getBoolVal().getValues()); - size = boolVars.length; - } else if (colValues.isSetByteVal()) { - type = Type.TINYINT_TYPE; - nulls = toBitset(colValues.getByteVal().getNulls()); - byteVars = Bytes.toArray(colValues.getByteVal().getValues()); - size = byteVars.length; - } else if (colValues.isSetI16Val()) { - type = Type.SMALLINT_TYPE; - nulls = toBitset(colValues.getI16Val().getNulls()); - shortVars = Shorts.toArray(colValues.getI16Val().getValues()); - size = shortVars.length; - } else if (colValues.isSetI32Val()) { - type = Type.INT_TYPE; - nulls = toBitset(colValues.getI32Val().getNulls()); - intVars = Ints.toArray(colValues.getI32Val().getValues()); - size = intVars.length; - } else if (colValues.isSetI64Val()) { - type = Type.BIGINT_TYPE; - nulls = toBitset(colValues.getI64Val().getNulls()); - longVars = Longs.toArray(colValues.getI64Val().getValues()); - size = longVars.length; - } else if (colValues.isSetDoubleVal()) { - type = Type.DOUBLE_TYPE; - nulls = toBitset(colValues.getDoubleVal().getNulls()); - doubleVars = Doubles.toArray(colValues.getDoubleVal().getValues()); - size = doubleVars.length; - } else if (colValues.isSetBinaryVal()) { - type = Type.BINARY_TYPE; - nulls = toBitset(colValues.getBinaryVal().getNulls()); - binaryVars = colValues.getBinaryVal().getValues(); - size = binaryVars.size(); - } else if (colValues.isSetStringVal()) { - type = Type.STRING_TYPE; - nulls = toBitset(colValues.getStringVal().getNulls()); - stringVars = colValues.getStringVal().getValues(); - size = stringVars.size(); - } else { - throw new IllegalStateException("invalid union object"); - } - } - - public Column extractSubset(int start, int end) { - BitSet subNulls = nulls.get(start, end); - if (type == Type.BOOLEAN_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(boolVars, start, end)); - boolVars = Arrays.copyOfRange(boolVars, end, size); - nulls = nulls.get(start, size); - size = boolVars.length; - return subset; - } - if (type == Type.TINYINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(byteVars, start, end)); - byteVars = Arrays.copyOfRange(byteVars, end, size); - nulls = nulls.get(start, size); - size = byteVars.length; - return subset; - } - if (type == Type.SMALLINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(shortVars, start, end)); - shortVars = Arrays.copyOfRange(shortVars, end, size); - nulls = nulls.get(start, size); - size = shortVars.length; - return subset; - } - if (type == Type.INT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(intVars, start, end)); - intVars = Arrays.copyOfRange(intVars, end, size); - nulls = nulls.get(start, size); - size = intVars.length; - return subset; - } - if (type == Type.BIGINT_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(longVars, start, end)); - longVars = Arrays.copyOfRange(longVars, end, size); - nulls = nulls.get(start, size); - size = longVars.length; - return subset; - } - if (type == Type.DOUBLE_TYPE) { - Column subset = new Column(type, subNulls, Arrays.copyOfRange(doubleVars, start, end)); - doubleVars = Arrays.copyOfRange(doubleVars, end, size); - nulls = nulls.get(start, size); - size = doubleVars.length; - return subset; - } - if (type == Type.BINARY_TYPE) { - Column subset = new Column(type, subNulls, binaryVars.subList(start, end)); - binaryVars = binaryVars.subList(end, binaryVars.size()); - nulls = nulls.get(start, size); - size = binaryVars.size(); - return subset; - } - if (type == Type.STRING_TYPE) { - Column subset = new Column(type, subNulls, stringVars.subList(start, end)); - stringVars = stringVars.subList(end, stringVars.size()); - nulls = nulls.get(start, size); - size = stringVars.size(); - return subset; - } - throw new IllegalStateException("invalid union object"); - } - - private static final byte[] MASKS = new byte[] { - 0x01, 0x02, 0x04, 0x08, 0x10, 0x20, 0x40, (byte)0x80 - }; - - private static BitSet toBitset(byte[] nulls) { - BitSet bitset = new BitSet(); - int bits = nulls.length * 8; - for (int i = 0; i < bits; i++) { - bitset.set(i, (nulls[i / 8] & MASKS[i % 8]) != 0); - } - return bitset; - } - - private static byte[] toBinary(BitSet bitset) { - byte[] nulls = new byte[1 + (bitset.length() / 8)]; - for (int i = 0; i < bitset.length(); i++) { - nulls[i / 8] |= bitset.get(i) ? MASKS[i % 8] : 0; - } - return nulls; - } - - public Type getType() { - return type; - } - - @Override - public Object get(int index) { - if (nulls.get(index)) { - return null; - } - switch (type) { - case BOOLEAN_TYPE: - return boolVars[index]; - case TINYINT_TYPE: - return byteVars[index]; - case SMALLINT_TYPE: - return shortVars[index]; - case INT_TYPE: - return intVars[index]; - case BIGINT_TYPE: - return longVars[index]; - case DOUBLE_TYPE: - return doubleVars[index]; - case STRING_TYPE: - return stringVars.get(index); - case BINARY_TYPE: - return binaryVars.get(index).array(); - } - return null; - } - - @Override - public int size() { - return size; - } - - public TColumn toTColumn() { - TColumn value = new TColumn(); - ByteBuffer nullMasks = ByteBuffer.wrap(toBinary(nulls)); - switch (type) { - case BOOLEAN_TYPE: - value.setBoolVal(new TBoolColumn(Booleans.asList(Arrays.copyOfRange(boolVars, 0, size)), nullMasks)); - break; - case TINYINT_TYPE: - value.setByteVal(new TByteColumn(Bytes.asList(Arrays.copyOfRange(byteVars, 0, size)), nullMasks)); - break; - case SMALLINT_TYPE: - value.setI16Val(new TI16Column(Shorts.asList(Arrays.copyOfRange(shortVars, 0, size)), nullMasks)); - break; - case INT_TYPE: - value.setI32Val(new TI32Column(Ints.asList(Arrays.copyOfRange(intVars, 0, size)), nullMasks)); - break; - case BIGINT_TYPE: - value.setI64Val(new TI64Column(Longs.asList(Arrays.copyOfRange(longVars, 0, size)), nullMasks)); - break; - case DOUBLE_TYPE: - value.setDoubleVal(new TDoubleColumn(Doubles.asList(Arrays.copyOfRange(doubleVars, 0, size)), nullMasks)); - break; - case STRING_TYPE: - value.setStringVal(new TStringColumn(stringVars, nullMasks)); - break; - case BINARY_TYPE: - value.setBinaryVal(new TBinaryColumn(binaryVars, nullMasks)); - break; - } - return value; - } - - private static final ByteBuffer EMPTY_BINARY = ByteBuffer.allocate(0); - private static final String EMPTY_STRING = ""; - - public void addValue(Type type, Object field) { - switch (type) { - case BOOLEAN_TYPE: - nulls.set(size, field == null); - boolVars()[size] = field == null ? true : (Boolean)field; - break; - case TINYINT_TYPE: - nulls.set(size, field == null); - byteVars()[size] = field == null ? 0 : (Byte) field; - break; - case SMALLINT_TYPE: - nulls.set(size, field == null); - shortVars()[size] = field == null ? 0 : (Short)field; - break; - case INT_TYPE: - nulls.set(size, field == null); - intVars()[size] = field == null ? 0 : (Integer)field; - break; - case BIGINT_TYPE: - nulls.set(size, field == null); - longVars()[size] = field == null ? 0 : (Long)field; - break; - case FLOAT_TYPE: - nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : Double.valueOf(field.toString()); - break; - case DOUBLE_TYPE: - nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : (Double)field; - break; - case BINARY_TYPE: - nulls.set(binaryVars.size(), field == null); - binaryVars.add(field == null ? EMPTY_BINARY : ByteBuffer.wrap((byte[])field)); - break; - default: - nulls.set(stringVars.size(), field == null); - stringVars.add(field == null ? EMPTY_STRING : String.valueOf(field)); - break; - } - size++; - } - - private boolean[] boolVars() { - if (boolVars.length == size) { - boolean[] newVars = new boolean[size << 1]; - System.arraycopy(boolVars, 0, newVars, 0, size); - return boolVars = newVars; - } - return boolVars; - } - - private byte[] byteVars() { - if (byteVars.length == size) { - byte[] newVars = new byte[size << 1]; - System.arraycopy(byteVars, 0, newVars, 0, size); - return byteVars = newVars; - } - return byteVars; - } - - private short[] shortVars() { - if (shortVars.length == size) { - short[] newVars = new short[size << 1]; - System.arraycopy(shortVars, 0, newVars, 0, size); - return shortVars = newVars; - } - return shortVars; - } - - private int[] intVars() { - if (intVars.length == size) { - int[] newVars = new int[size << 1]; - System.arraycopy(intVars, 0, newVars, 0, size); - return intVars = newVars; - } - return intVars; - } - - private long[] longVars() { - if (longVars.length == size) { - long[] newVars = new long[size << 1]; - System.arraycopy(longVars, 0, newVars, 0, size); - return longVars = newVars; - } - return longVars; - } - - private double[] doubleVars() { - if (doubleVars.length == size) { - double[] newVars = new double[size << 1]; - System.arraycopy(doubleVars, 0, newVars, 0, size); - return doubleVars = newVars; - } - return doubleVars; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java deleted file mode 100644 index 47a582e2223e4..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java +++ /dev/null @@ -1,149 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TColumn; -import org.apache.hive.service.cli.thrift.TRow; -import org.apache.hive.service.cli.thrift.TRowSet; - -/** - * ColumnBasedSet. - */ -public class ColumnBasedSet implements RowSet { - - private long startOffset; - - private final Type[] types; // non-null only for writing (server-side) - private final List columns; - - public ColumnBasedSet(TableSchema schema) { - types = schema.toTypes(); - columns = new ArrayList(); - for (ColumnDescriptor colDesc : schema.getColumnDescriptors()) { - columns.add(new Column(colDesc.getType())); - } - } - - public ColumnBasedSet(TRowSet tRowSet) { - types = null; - columns = new ArrayList(); - for (TColumn tvalue : tRowSet.getColumns()) { - columns.add(new Column(tvalue)); - } - startOffset = tRowSet.getStartRowOffset(); - } - - private ColumnBasedSet(Type[] types, List columns, long startOffset) { - this.types = types; - this.columns = columns; - this.startOffset = startOffset; - } - - @Override - public ColumnBasedSet addRow(Object[] fields) { - for (int i = 0; i < fields.length; i++) { - columns.get(i).addValue(types[i], fields[i]); - } - return this; - } - - public List getColumns() { - return columns; - } - - @Override - public int numColumns() { - return columns.size(); - } - - @Override - public int numRows() { - return columns.isEmpty() ? 0 : columns.get(0).size(); - } - - @Override - public ColumnBasedSet extractSubset(int maxRows) { - int numRows = Math.min(numRows(), maxRows); - - List subset = new ArrayList(); - for (int i = 0; i < columns.size(); i++) { - subset.add(columns.get(i).extractSubset(0, numRows)); - } - ColumnBasedSet result = new ColumnBasedSet(types, subset, startOffset); - startOffset += numRows; - return result; - } - - @Override - public long getStartOffset() { - return startOffset; - } - - @Override - public void setStartOffset(long startOffset) { - this.startOffset = startOffset; - } - - public TRowSet toTRowSet() { - TRowSet tRowSet = new TRowSet(startOffset, new ArrayList()); - for (int i = 0; i < columns.size(); i++) { - tRowSet.addToColumns(columns.get(i).toTColumn()); - } - return tRowSet; - } - - @Override - public Iterator iterator() { - return new Iterator() { - - private int index; - private final Object[] convey = new Object[numColumns()]; - - @Override - public boolean hasNext() { - return index < numRows(); - } - - @Override - public Object[] next() { - for (int i = 0; i < columns.size(); i++) { - convey[i] = columns.get(i).get(index); - } - index++; - return convey; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove"); - } - }; - } - - public Object[] fill(int index, Object[] convey) { - for (int i = 0; i < columns.size(); i++) { - convey[i] = columns.get(i).get(index); - } - return convey; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java deleted file mode 100644 index f0bbf14693160..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java +++ /dev/null @@ -1,99 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hive.service.cli.thrift.TColumnDesc; - - -/** - * ColumnDescriptor. - * - */ -public class ColumnDescriptor { - private final String name; - private final String comment; - private final TypeDescriptor type; - // ordinal position of this column in the schema - private final int position; - - public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) { - this.name = name; - this.comment = comment; - this.type = type; - this.position = position; - } - - public ColumnDescriptor(TColumnDesc tColumnDesc) { - name = tColumnDesc.getColumnName(); - comment = tColumnDesc.getComment(); - type = new TypeDescriptor(tColumnDesc.getTypeDesc()); - position = tColumnDesc.getPosition(); - } - - public ColumnDescriptor(FieldSchema column, int position) { - name = column.getName(); - comment = column.getComment(); - type = new TypeDescriptor(column.getType()); - this.position = position; - } - - public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) { - // Current usage looks like it's only for metadata columns, but if that changes then - // this method may need to require a type qualifiers aruments. - return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position); - } - - public String getName() { - return name; - } - - public String getComment() { - return comment; - } - - public TypeDescriptor getTypeDescriptor() { - return type; - } - - public int getOrdinalPosition() { - return position; - } - - public TColumnDesc toTColumnDesc() { - TColumnDesc tColumnDesc = new TColumnDesc(); - tColumnDesc.setColumnName(name); - tColumnDesc.setComment(comment); - tColumnDesc.setTypeDesc(type.toTTypeDesc()); - tColumnDesc.setPosition(position); - return tColumnDesc; - } - - public Type getType() { - return type.getType(); - } - - public boolean isPrimitive() { - return type.getType().isPrimitiveType(); - } - - public String getTypeName() { - return type.getTypeName(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java deleted file mode 100644 index 462b93a0f09fe..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java +++ /dev/null @@ -1,288 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Timestamp; - -import org.apache.hadoop.hive.common.type.HiveChar; -import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; -import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; -import org.apache.hadoop.hive.common.type.HiveVarchar; -import org.apache.hive.service.cli.thrift.TBoolValue; -import org.apache.hive.service.cli.thrift.TByteValue; -import org.apache.hive.service.cli.thrift.TColumnValue; -import org.apache.hive.service.cli.thrift.TDoubleValue; -import org.apache.hive.service.cli.thrift.TI16Value; -import org.apache.hive.service.cli.thrift.TI32Value; -import org.apache.hive.service.cli.thrift.TI64Value; -import org.apache.hive.service.cli.thrift.TStringValue; - -import org.apache.spark.unsafe.types.UTF8String; - -/** - * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) - * - */ -public class ColumnValue { - - private static TColumnValue booleanValue(Boolean value) { - TBoolValue tBoolValue = new TBoolValue(); - if (value != null) { - tBoolValue.setValue(value); - } - return TColumnValue.boolVal(tBoolValue); - } - - private static TColumnValue byteValue(Byte value) { - TByteValue tByteValue = new TByteValue(); - if (value != null) { - tByteValue.setValue(value); - } - return TColumnValue.byteVal(tByteValue); - } - - private static TColumnValue shortValue(Short value) { - TI16Value tI16Value = new TI16Value(); - if (value != null) { - tI16Value.setValue(value); - } - return TColumnValue.i16Val(tI16Value); - } - - private static TColumnValue intValue(Integer value) { - TI32Value tI32Value = new TI32Value(); - if (value != null) { - tI32Value.setValue(value); - } - return TColumnValue.i32Val(tI32Value); - } - - private static TColumnValue longValue(Long value) { - TI64Value tI64Value = new TI64Value(); - if (value != null) { - tI64Value.setValue(value); - } - return TColumnValue.i64Val(tI64Value); - } - - private static TColumnValue floatValue(Float value) { - TDoubleValue tDoubleValue = new TDoubleValue(); - if (value != null) { - tDoubleValue.setValue(value); - } - return TColumnValue.doubleVal(tDoubleValue); - } - - private static TColumnValue doubleValue(Double value) { - TDoubleValue tDoubleValue = new TDoubleValue(); - if (value != null) { - tDoubleValue.setValue(value); - } - return TColumnValue.doubleVal(tDoubleValue); - } - - private static TColumnValue stringValue(String value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveChar value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveVarchar value) { - TStringValue tStringValue = new TStringValue(); - if (value != null) { - tStringValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStringValue); - } - - private static TColumnValue stringValue(HiveIntervalYearMonth value) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - tStrValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStrValue); - } - - private static TColumnValue stringValue(HiveIntervalDayTime value) { - TStringValue tStrValue = new TStringValue(); - if (value != null) { - tStrValue.setValue(value.toString()); - } - return TColumnValue.stringVal(tStrValue); - } - - public static TColumnValue toTColumnValue(Type type, Object value) { - switch (type) { - case BOOLEAN_TYPE: - return booleanValue((Boolean)value); - case TINYINT_TYPE: - return byteValue((Byte)value); - case SMALLINT_TYPE: - return shortValue((Short)value); - case INT_TYPE: - return intValue((Integer)value); - case BIGINT_TYPE: - return longValue((Long)value); - case FLOAT_TYPE: - return floatValue((Float)value); - case DOUBLE_TYPE: - return doubleValue((Double)value); - case STRING_TYPE: - return stringValue((String)value); - case CHAR_TYPE: - return stringValue((HiveChar)value); - case VARCHAR_TYPE: - return stringValue((HiveVarchar)value); - case DATE_TYPE: - case TIMESTAMP_TYPE: - // SPARK-31859, SPARK-31861: converted to string already in SparkExecuteStatementOperation - return stringValue((String)value); - case INTERVAL_YEAR_MONTH_TYPE: - return stringValue((HiveIntervalYearMonth) value); - case INTERVAL_DAY_TIME_TYPE: - return stringValue((HiveIntervalDayTime) value); - case DECIMAL_TYPE: - String plainStr = value == null ? null : ((BigDecimal)value).toPlainString(); - return stringValue(plainStr); - case BINARY_TYPE: - String strVal = value == null ? null : UTF8String.fromBytes((byte[])value).toString(); - return stringValue(strVal); - case ARRAY_TYPE: - case MAP_TYPE: - case STRUCT_TYPE: - case UNION_TYPE: - case USER_DEFINED_TYPE: - return stringValue((String)value); - case NULL_TYPE: - return stringValue((String)value); - default: - return null; - } - } - - private static Boolean getBooleanValue(TBoolValue tBoolValue) { - if (tBoolValue.isSetValue()) { - return tBoolValue.isValue(); - } - return null; - } - - private static Byte getByteValue(TByteValue tByteValue) { - if (tByteValue.isSetValue()) { - return tByteValue.getValue(); - } - return null; - } - - private static Short getShortValue(TI16Value tI16Value) { - if (tI16Value.isSetValue()) { - return tI16Value.getValue(); - } - return null; - } - - private static Integer getIntegerValue(TI32Value tI32Value) { - if (tI32Value.isSetValue()) { - return tI32Value.getValue(); - } - return null; - } - - private static Long getLongValue(TI64Value tI64Value) { - if (tI64Value.isSetValue()) { - return tI64Value.getValue(); - } - return null; - } - - private static Double getDoubleValue(TDoubleValue tDoubleValue) { - if (tDoubleValue.isSetValue()) { - return tDoubleValue.getValue(); - } - return null; - } - - private static String getStringValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return tStringValue.getValue(); - } - return null; - } - - private static Date getDateValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return Date.valueOf(tStringValue.getValue()); - } - return null; - } - - private static Timestamp getTimestampValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return Timestamp.valueOf(tStringValue.getValue()); - } - return null; - } - - private static byte[] getBinaryValue(TStringValue tString) { - if (tString.isSetValue()) { - return tString.getValue().getBytes(); - } - return null; - } - - private static BigDecimal getBigDecimalValue(TStringValue tStringValue) { - if (tStringValue.isSetValue()) { - return new BigDecimal(tStringValue.getValue()); - } - return null; - } - - public static Object toColumnValue(TColumnValue value) { - TColumnValue._Fields field = value.getSetField(); - switch (field) { - case BOOL_VAL: - return getBooleanValue(value.getBoolVal()); - case BYTE_VAL: - return getByteValue(value.getByteVal()); - case I16_VAL: - return getShortValue(value.getI16Val()); - case I32_VAL: - return getIntegerValue(value.getI32Val()); - case I64_VAL: - return getLongValue(value.getI64Val()); - case DOUBLE_VAL: - return getDoubleValue(value.getDoubleVal()); - case STRING_VAL: - return getStringValue(value.getStringVal()); - } - throw new IllegalArgumentException("never"); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java deleted file mode 100644 index 9cad5be198c06..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java +++ /dev/null @@ -1,208 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.List; -import java.util.Map; - -import org.apache.hive.service.auth.HiveAuthFactory; - - -/** - * EmbeddedCLIServiceClient. - * - */ -public class EmbeddedCLIServiceClient extends CLIServiceClient { - private final ICLIService cliService; - - public EmbeddedCLIServiceClient(ICLIService cliService) { - this.cliService = cliService; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, - Map configuration) throws HiveSQLException { - return cliService.openSession(username, password, configuration); - } - - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) throws HiveSQLException { - throw new HiveSQLException("Impersonated session is not supported in the embedded mode"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - cliService.closeSession(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) - throws HiveSQLException { - return cliService.getInfo(sessionHandle, getInfoType); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - return cliService.executeStatement(sessionHandle, statement, confOverlay); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, - * java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) throws HiveSQLException { - return cliService.executeStatementAsync(sessionHandle, statement, confOverlay); - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getTypeInfo(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getCatalogs(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, - String schemaName) throws HiveSQLException { - return cliService.getSchemas(sessionHandle, catalogName, schemaName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, List tableTypes) throws HiveSQLException { - return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { - return cliService.getTableTypes(sessionHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, String columnName) throws HiveSQLException { - return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException { - return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { - return cliService.getOperationStatus(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - cliService.cancelOperation(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - cliService.closeOperation(opHandle); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.CLIServiceClient#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { - return cliService.getResultSetMetadata(opHandle); - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - return cliService.fetchResults(opHandle, orientation, maxRows, fetchType); - } - - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer); - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr); - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java deleted file mode 100644 index ffa6f2e1f3743..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TFetchOrientation; - -/** - * FetchOrientation. - * - */ -public enum FetchOrientation { - FETCH_NEXT(TFetchOrientation.FETCH_NEXT), - FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR), - FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE), - FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE), - FETCH_FIRST(TFetchOrientation.FETCH_FIRST), - FETCH_LAST(TFetchOrientation.FETCH_LAST); - - private TFetchOrientation tFetchOrientation; - - FetchOrientation(TFetchOrientation tFetchOrientation) { - this.tFetchOrientation = tFetchOrientation; - } - - public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) { - for (FetchOrientation fetchOrientation : values()) { - if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) { - return fetchOrientation; - } - } - // TODO: Should this really default to FETCH_NEXT? - return FETCH_NEXT; - } - - public TFetchOrientation toTFetchOrientation() { - return tFetchOrientation; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java deleted file mode 100644 index 8dd33a88fdeb2..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TGetInfoType; - -/** - * GetInfoType. - * - */ -public enum GetInfoType { - CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS), - CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES), - CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME), - CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION), - CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME), - CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE), - CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME), - CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER), - CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES), - CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES), - CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR), - CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY), - CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION), - CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE), - CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR), - CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN), - CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN), - CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN), - CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN), - CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN), - CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY), - CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE), - CLI_USER_NAME(TGetInfoType.CLI_USER_NAME), - CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION), - CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY), - CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS), - CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION), - CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE), - CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT), - CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS), - CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY), - CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX), - CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY), - CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT), - CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE), - CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE), - CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE), - CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN), - CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT), - CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN), - CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES), - - CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR), - CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY), - CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER), - CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME), - CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ), - CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN); - - private final TGetInfoType tInfoType; - - GetInfoType(TGetInfoType tInfoType) { - this.tInfoType = tInfoType; - } - - public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) { - for (GetInfoType infoType : values()) { - if (tGetInfoType.equals(infoType.tInfoType)) { - return infoType; - } - } - throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType); - } - - public TGetInfoType toTGetInfoType() { - return tInfoType; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java deleted file mode 100644 index ba92ff4ab5c11..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TGetInfoValue; - -/** - * GetInfoValue. - * - */ -public class GetInfoValue { - private String stringValue = null; - private short shortValue; - private int intValue; - private long longValue; - - public GetInfoValue(String stringValue) { - this.stringValue = stringValue; - } - - public GetInfoValue(short shortValue) { - this.shortValue = shortValue; - } - - public GetInfoValue(int intValue) { - this.intValue = intValue; - } - - public GetInfoValue(long longValue) { - this.longValue = longValue; - } - - public GetInfoValue(TGetInfoValue tGetInfoValue) { - switch (tGetInfoValue.getSetField()) { - case STRING_VALUE: - stringValue = tGetInfoValue.getStringValue(); - break; - default: - throw new IllegalArgumentException("Unreconigzed TGetInfoValue"); - } - } - - public TGetInfoValue toTGetInfoValue() { - TGetInfoValue tInfoValue = new TGetInfoValue(); - if (stringValue != null) { - tInfoValue.setStringValue(stringValue); - } - return tInfoValue; - } - - public String getStringValue() { - return stringValue; - } - - public short getShortValue() { - return shortValue; - } - - public int getIntValue() { - return intValue; - } - - public long getLongValue() { - return longValue; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java deleted file mode 100644 index cf3427ae20f3c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.THandleIdentifier; - - - - -public abstract class Handle { - - private final HandleIdentifier handleId; - - public Handle() { - handleId = new HandleIdentifier(); - } - - public Handle(HandleIdentifier handleId) { - this.handleId = handleId; - } - - public Handle(THandleIdentifier tHandleIdentifier) { - this.handleId = new HandleIdentifier(tHandleIdentifier); - } - - public HandleIdentifier getHandleIdentifier() { - return handleId; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((handleId == null) ? 0 : handleId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof Handle)) { - return false; - } - Handle other = (Handle) obj; - if (handleId == null) { - if (other.handleId != null) { - return false; - } - } else if (!handleId.equals(other.handleId)) { - return false; - } - return true; - } - - @Override - public abstract String toString(); - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java deleted file mode 100644 index 4dc80da8dc500..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.nio.ByteBuffer; -import java.util.UUID; - -import org.apache.hive.service.cli.thrift.THandleIdentifier; - -/** - * HandleIdentifier. - * - */ -public class HandleIdentifier { - private final UUID publicId; - private final UUID secretId; - - public HandleIdentifier() { - publicId = UUID.randomUUID(); - secretId = UUID.randomUUID(); - } - - public HandleIdentifier(UUID publicId, UUID secretId) { - this.publicId = publicId; - this.secretId = secretId; - } - - public HandleIdentifier(THandleIdentifier tHandleId) { - ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid()); - this.publicId = new UUID(bb.getLong(), bb.getLong()); - bb = ByteBuffer.wrap(tHandleId.getSecret()); - this.secretId = new UUID(bb.getLong(), bb.getLong()); - } - - public UUID getPublicId() { - return publicId; - } - - public UUID getSecretId() { - return secretId; - } - - public THandleIdentifier toTHandleIdentifier() { - byte[] guid = new byte[16]; - byte[] secret = new byte[16]; - ByteBuffer guidBB = ByteBuffer.wrap(guid); - ByteBuffer secretBB = ByteBuffer.wrap(secret); - guidBB.putLong(publicId.getMostSignificantBits()); - guidBB.putLong(publicId.getLeastSignificantBits()); - secretBB.putLong(secretId.getMostSignificantBits()); - secretBB.putLong(secretId.getLeastSignificantBits()); - return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret)); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((publicId == null) ? 0 : publicId.hashCode()); - result = prime * result + ((secretId == null) ? 0 : secretId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof HandleIdentifier)) { - return false; - } - HandleIdentifier other = (HandleIdentifier) obj; - if (publicId == null) { - if (other.publicId != null) { - return false; - } - } else if (!publicId.equals(other.publicId)) { - return false; - } - if (secretId == null) { - if (other.secretId != null) { - return false; - } - } else if (!secretId.equals(other.secretId)) { - return false; - } - return true; - } - - @Override - public String toString() { - return publicId.toString(); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java deleted file mode 100644 index 86e57fbf31fe0..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java +++ /dev/null @@ -1,249 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TStatus; -import org.apache.hive.service.cli.thrift.TStatusCode; - -/** - * HiveSQLException. - * - */ -public class HiveSQLException extends SQLException { - - /** - * - */ - private static final long serialVersionUID = -6095254671958748094L; - - /** - * - */ - public HiveSQLException() { - super(); - } - - /** - * @param reason - */ - public HiveSQLException(String reason) { - super(reason); - } - - /** - * @param cause - */ - public HiveSQLException(Throwable cause) { - super(cause); - } - - /** - * @param reason - * @param sqlState - */ - public HiveSQLException(String reason, String sqlState) { - super(reason, sqlState); - } - - /** - * @param reason - * @param cause - */ - public HiveSQLException(String reason, Throwable cause) { - super(reason, cause); - } - - /** - * @param reason - * @param sqlState - * @param vendorCode - */ - public HiveSQLException(String reason, String sqlState, int vendorCode) { - super(reason, sqlState, vendorCode); - } - - /** - * @param reason - * @param sqlState - * @param cause - */ - public HiveSQLException(String reason, String sqlState, Throwable cause) { - super(reason, sqlState, cause); - } - - /** - * @param reason - * @param sqlState - * @param vendorCode - * @param cause - */ - public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) { - super(reason, sqlState, vendorCode, cause); - } - - public HiveSQLException(TStatus status) { - // TODO: set correct vendorCode field - super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode()); - if (status.getInfoMessages() != null) { - initCause(toCause(status.getInfoMessages())); - } - } - - /** - * Converts current object to a {@link TStatus} object - * @return a {@link TStatus} object - */ - public TStatus toTStatus() { - // TODO: convert sqlState, etc. - TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); - tStatus.setSqlState(getSQLState()); - tStatus.setErrorCode(getErrorCode()); - tStatus.setErrorMessage(getMessage()); - tStatus.setInfoMessages(toString(this)); - return tStatus; - } - - /** - * Converts the specified {@link Exception} object into a {@link TStatus} object - * @param e a {@link Exception} object - * @return a {@link TStatus} object - */ - public static TStatus toTStatus(Exception e) { - if (e instanceof HiveSQLException) { - return ((HiveSQLException)e).toTStatus(); - } - TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); - tStatus.setErrorMessage(e.getMessage()); - tStatus.setInfoMessages(toString(e)); - return tStatus; - } - - /** - * Converts a {@link Throwable} object into a flattened list of texts including its stack trace - * and the stack traces of the nested causes. - * @param ex a {@link Throwable} object - * @return a flattened list of texts including the {@link Throwable} object's stack trace - * and the stack traces of the nested causes. - */ - public static List toString(Throwable ex) { - return toString(ex, null); - } - - private static List toString(Throwable cause, StackTraceElement[] parent) { - StackTraceElement[] trace = cause.getStackTrace(); - int m = trace.length - 1; - if (parent != null) { - int n = parent.length - 1; - while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { - m--; - n--; - } - } - List detail = enroll(cause, trace, m); - cause = cause.getCause(); - if (cause != null) { - detail.addAll(toString(cause, trace)); - } - return detail; - } - - private static List enroll(Throwable ex, StackTraceElement[] trace, int max) { - List details = new ArrayList(); - StringBuilder builder = new StringBuilder(); - builder.append('*').append(ex.getClass().getName()).append(':'); - builder.append(ex.getMessage()).append(':'); - builder.append(trace.length).append(':').append(max); - details.add(builder.toString()); - for (int i = 0; i <= max; i++) { - builder.setLength(0); - builder.append(trace[i].getClassName()).append(':'); - builder.append(trace[i].getMethodName()).append(':'); - String fileName = trace[i].getFileName(); - builder.append(fileName == null ? "" : fileName).append(':'); - builder.append(trace[i].getLineNumber()); - details.add(builder.toString()); - } - return details; - } - - /** - * Converts a flattened list of texts including the stack trace and the stack - * traces of the nested causes into a {@link Throwable} object. - * @param details a flattened list of texts including the stack trace and the stack - * traces of the nested causes - * @return a {@link Throwable} object - */ - public static Throwable toCause(List details) { - return toStackTrace(details, null, 0); - } - - private static Throwable toStackTrace(List details, StackTraceElement[] parent, int index) { - String detail = details.get(index++); - if (!detail.startsWith("*")) { - return null; // should not be happened. ignore remaining - } - int i1 = detail.indexOf(':'); - int i3 = detail.lastIndexOf(':'); - int i2 = detail.substring(0, i3).lastIndexOf(':'); - String exceptionClass = detail.substring(1, i1); - String exceptionMessage = detail.substring(i1 + 1, i2); - Throwable ex = newInstance(exceptionClass, exceptionMessage); - - Integer length = Integer.valueOf(detail.substring(i2 + 1, i3)); - Integer unique = Integer.valueOf(detail.substring(i3 + 1)); - - int i = 0; - StackTraceElement[] trace = new StackTraceElement[length]; - for (; i <= unique; i++) { - detail = details.get(index++); - int j1 = detail.indexOf(':'); - int j3 = detail.lastIndexOf(':'); - int j2 = detail.substring(0, j3).lastIndexOf(':'); - String className = detail.substring(0, j1); - String methodName = detail.substring(j1 + 1, j2); - String fileName = detail.substring(j2 + 1, j3); - if (fileName.isEmpty()) { - fileName = null; - } - int lineNumber = Integer.valueOf(detail.substring(j3 + 1)); - trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber); - } - int common = trace.length - i; - if (common > 0) { - System.arraycopy(parent, parent.length - common, trace, trace.length - common, common); - } - if (details.size() > index) { - ex.initCause(toStackTrace(details, trace, index)); - } - ex.setStackTrace(trace); - return ex; - } - - private static Throwable newInstance(String className, String message) { - try { - return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message); - } catch (Exception e) { - return new RuntimeException(className + ":" + message); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java deleted file mode 100644 index c9cc1f4da56f1..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.List; -import java.util.Map; - - - - -import org.apache.hive.service.auth.HiveAuthFactory; - -public interface ICLIService { - - SessionHandle openSession(String username, String password, - Map configuration) - throws HiveSQLException; - - SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) - throws HiveSQLException; - - void closeSession(SessionHandle sessionHandle) - throws HiveSQLException; - - GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) - throws HiveSQLException; - - OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException; - - OperationHandle executeStatementAsync(SessionHandle sessionHandle, - String statement, Map confOverlay) - throws HiveSQLException; - - OperationHandle getTypeInfo(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getCatalogs(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getSchemas(SessionHandle sessionHandle, - String catalogName, String schemaName) - throws HiveSQLException; - - OperationHandle getTables(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, List tableTypes) - throws HiveSQLException; - - OperationHandle getTableTypes(SessionHandle sessionHandle) - throws HiveSQLException; - - OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException; - - OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) - throws HiveSQLException; - - OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException; - - void cancelOperation(OperationHandle opHandle) - throws HiveSQLException; - - void closeOperation(OperationHandle opHandle) - throws HiveSQLException; - - TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException; - - String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException; - - void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException; - - void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException; - - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java deleted file mode 100644 index 5426e28471239..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationHandle; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -public class OperationHandle extends Handle { - - private final OperationType opType; - private final TProtocolVersion protocol; - private boolean hasResultSet = false; - - public OperationHandle(OperationType opType, TProtocolVersion protocol) { - super(); - this.opType = opType; - this.protocol = protocol; - } - - // dummy handle for ThriftCLIService - public OperationHandle(TOperationHandle tOperationHandle) { - this(tOperationHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); - } - - public OperationHandle(TOperationHandle tOperationHandle, TProtocolVersion protocol) { - super(tOperationHandle.getOperationId()); - this.opType = OperationType.getOperationType(tOperationHandle.getOperationType()); - this.hasResultSet = tOperationHandle.isHasResultSet(); - this.protocol = protocol; - } - - public OperationType getOperationType() { - return opType; - } - - public void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - } - - public boolean hasResultSet() { - return hasResultSet; - } - - public TOperationHandle toTOperationHandle() { - TOperationHandle tOperationHandle = new TOperationHandle(); - tOperationHandle.setOperationId(getHandleIdentifier().toTHandleIdentifier()); - tOperationHandle.setOperationType(opType.toTOperationType()); - tOperationHandle.setHasResultSet(hasResultSet); - return tOperationHandle; - } - - public TProtocolVersion getProtocolVersion() { - return protocol; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = super.hashCode(); - result = prime * result + ((opType == null) ? 0 : opType.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!super.equals(obj)) { - return false; - } - if (!(obj instanceof OperationHandle)) { - return false; - } - OperationHandle other = (OperationHandle) obj; - if (opType != other.opType) { - return false; - } - return true; - } - - @Override - public String toString() { - return "OperationHandle [opType=" + opType + ", getHandleIdentifier()=" + getHandleIdentifier() - + "]"; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java deleted file mode 100644 index 1165180118413..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationState; - -/** - * OperationState. - * - */ -public enum OperationState { - INITIALIZED(TOperationState.INITIALIZED_STATE, false), - RUNNING(TOperationState.RUNNING_STATE, false), - FINISHED(TOperationState.FINISHED_STATE, true), - CANCELED(TOperationState.CANCELED_STATE, true), - CLOSED(TOperationState.CLOSED_STATE, true), - ERROR(TOperationState.ERROR_STATE, true), - UNKNOWN(TOperationState.UKNOWN_STATE, false), - PENDING(TOperationState.PENDING_STATE, false); - - private final TOperationState tOperationState; - private final boolean terminal; - - OperationState(TOperationState tOperationState, boolean terminal) { - this.tOperationState = tOperationState; - this.terminal = terminal; - } - - // must be sync with TOperationState in order - public static OperationState getOperationState(TOperationState tOperationState) { - return OperationState.values()[tOperationState.getValue()]; - } - - public static void validateTransition(OperationState oldState, - OperationState newState) - throws HiveSQLException { - switch (oldState) { - case INITIALIZED: - switch (newState) { - case PENDING: - case RUNNING: - case CANCELED: - case CLOSED: - return; - } - break; - case PENDING: - switch (newState) { - case RUNNING: - case FINISHED: - case CANCELED: - case ERROR: - case CLOSED: - return; - } - break; - case RUNNING: - switch (newState) { - case FINISHED: - case CANCELED: - case ERROR: - case CLOSED: - return; - } - break; - case FINISHED: - case CANCELED: - case ERROR: - if (OperationState.CLOSED.equals(newState)) { - return; - } - break; - default: - // fall-through - } - throw new HiveSQLException("Illegal Operation state transition " + - "from " + oldState + " to " + newState); - } - - public void validateTransition(OperationState newState) - throws HiveSQLException { - validateTransition(this, newState); - } - - public TOperationState toTOperationState() { - return tOperationState; - } - - public boolean isTerminal() { - return terminal; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java deleted file mode 100644 index 429d9a4c25688..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TOperationType; - -/** - * OperationType. - * - */ -public enum OperationType { - - UNKNOWN_OPERATION(TOperationType.UNKNOWN), - EXECUTE_STATEMENT(TOperationType.EXECUTE_STATEMENT), - GET_TYPE_INFO(TOperationType.GET_TYPE_INFO), - GET_CATALOGS(TOperationType.GET_CATALOGS), - GET_SCHEMAS(TOperationType.GET_SCHEMAS), - GET_TABLES(TOperationType.GET_TABLES), - GET_TABLE_TYPES(TOperationType.GET_TABLE_TYPES), - GET_COLUMNS(TOperationType.GET_COLUMNS), - GET_FUNCTIONS(TOperationType.GET_FUNCTIONS); - - private TOperationType tOperationType; - - OperationType(TOperationType tOpType) { - this.tOperationType = tOpType; - } - - public static OperationType getOperationType(TOperationType tOperationType) { - // TODO: replace this with a Map? - for (OperationType opType : values()) { - if (tOperationType.equals(opType.tOperationType)) { - return opType; - } - } - return OperationType.UNKNOWN_OPERATION; - } - - public TOperationType toTOperationType() { - return tOperationType; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java deleted file mode 100644 index 6e4d43fd5df63..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * 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.hive.service.cli; - -/** - * PatternOrIdentifier. - * - */ -public class PatternOrIdentifier { - - boolean isPattern = false; - String text; - - public PatternOrIdentifier(String tpoi) { - text = tpoi; - isPattern = false; - } - - public boolean isPattern() { - return isPattern; - } - - public boolean isIdentifier() { - return !isPattern; - } - - @Override - public String toString() { - return text; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java deleted file mode 100644 index 7452137f077db..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ /dev/null @@ -1,140 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.hive.service.cli.thrift.TColumnValue; -import org.apache.hive.service.cli.thrift.TRow; -import org.apache.hive.service.cli.thrift.TRowSet; - -/** - * RowBasedSet - */ -public class RowBasedSet implements RowSet { - - private long startOffset; - - private final Type[] types; // non-null only for writing (server-side) - private final RemovableList rows; - - public RowBasedSet(TableSchema schema) { - types = schema.toTypes(); - rows = new RemovableList(); - } - - public RowBasedSet(TRowSet tRowSet) { - types = null; - rows = new RemovableList(tRowSet.getRows()); - startOffset = tRowSet.getStartRowOffset(); - } - - private RowBasedSet(Type[] types, List rows, long startOffset) { - this.types = types; - this.rows = new RemovableList(rows); - this.startOffset = startOffset; - } - - @Override - public RowBasedSet addRow(Object[] fields) { - TRow tRow = new TRow(); - for (int i = 0; i < fields.length; i++) { - tRow.addToColVals(ColumnValue.toTColumnValue(types[i], fields[i])); - } - rows.add(tRow); - return this; - } - - @Override - public int numColumns() { - return rows.isEmpty() ? 0 : rows.get(0).getColVals().size(); - } - - @Override - public int numRows() { - return rows.size(); - } - - public RowBasedSet extractSubset(int maxRows) { - int numRows = Math.min(numRows(), maxRows); - RowBasedSet result = new RowBasedSet(types, rows.subList(0, numRows), startOffset); - rows.removeRange(0, numRows); - startOffset += numRows; - return result; - } - - public long getStartOffset() { - return startOffset; - } - - public void setStartOffset(long startOffset) { - this.startOffset = startOffset; - } - - public int getSize() { - return rows.size(); - } - - public TRowSet toTRowSet() { - TRowSet tRowSet = new TRowSet(); - tRowSet.setStartRowOffset(startOffset); - tRowSet.setRows(new ArrayList(rows)); - return tRowSet; - } - - @Override - public Iterator iterator() { - return new Iterator() { - - final Iterator iterator = rows.iterator(); - final Object[] convey = new Object[numColumns()]; - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public Object[] next() { - TRow row = iterator.next(); - List values = row.getColVals(); - for (int i = 0; i < values.size(); i++) { - convey[i] = ColumnValue.toColumnValue(values.get(i)); - } - return convey; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove"); - } - }; - } - - private static class RemovableList extends ArrayList { - RemovableList() { super(); } - RemovableList(List rows) { super(rows); } - @Override - public void removeRange(int fromIndex, int toIndex) { - super.removeRange(fromIndex, toIndex); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java deleted file mode 100644 index ab0787e1d389e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java +++ /dev/null @@ -1,38 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TRowSet; - -public interface RowSet extends Iterable { - - RowSet addRow(Object[] fields); - - RowSet extractSubset(int maxRows); - - int numColumns(); - - int numRows(); - - long getStartOffset(); - - void setStartOffset(long startOffset); - - TRowSet toTRowSet(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java deleted file mode 100644 index e8f68eaaf9063..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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.hive.service.cli; - -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.cli.thrift.TRowSet; - -import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; - -public class RowSetFactory { - - public static RowSet create(TableSchema schema, TProtocolVersion version) { - if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { - return new ColumnBasedSet(schema); - } - return new RowBasedSet(schema); - } - - public static RowSet create(TRowSet results, TProtocolVersion version) { - if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { - return new ColumnBasedSet(results); - } - return new RowBasedSet(results); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java deleted file mode 100644 index 52e0ad4834d8b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.UUID; - -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.cli.thrift.TSessionHandle; - - -/** - * SessionHandle. - * - */ -public class SessionHandle extends Handle { - - private final TProtocolVersion protocol; - - public SessionHandle(TProtocolVersion protocol) { - this.protocol = protocol; - } - - // dummy handle for ThriftCLIService - public SessionHandle(TSessionHandle tSessionHandle) { - this(tSessionHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); - } - - public SessionHandle(TSessionHandle tSessionHandle, TProtocolVersion protocol) { - super(tSessionHandle.getSessionId()); - this.protocol = protocol; - } - - public UUID getSessionId() { - return getHandleIdentifier().getPublicId(); - } - - public TSessionHandle toTSessionHandle() { - TSessionHandle tSessionHandle = new TSessionHandle(); - tSessionHandle.setSessionId(getHandleIdentifier().toTHandleIdentifier()); - return tSessionHandle; - } - - public TProtocolVersion getProtocolVersion() { - return protocol; - } - - @Override - public String toString() { - return "SessionHandle [" + getHandleIdentifier() + "]"; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java deleted file mode 100644 index ee019bc737101..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hive.service.cli.thrift.TColumnDesc; -import org.apache.hive.service.cli.thrift.TTableSchema; - -/** - * TableSchema. - * - */ -public class TableSchema { - private final List columns = new ArrayList(); - - public TableSchema() { - } - - public TableSchema(int numColumns) { - // TODO: remove this constructor - } - - public TableSchema(TTableSchema tTableSchema) { - for (TColumnDesc tColumnDesc : tTableSchema.getColumns()) { - columns.add(new ColumnDescriptor(tColumnDesc)); - } - } - - public TableSchema(List fieldSchemas) { - int pos = 1; - for (FieldSchema field : fieldSchemas) { - columns.add(new ColumnDescriptor(field, pos++)); - } - } - - public TableSchema(Schema schema) { - this(schema.getFieldSchemas()); - } - - public List getColumnDescriptors() { - return new ArrayList(columns); - } - - public ColumnDescriptor getColumnDescriptorAt(int pos) { - return columns.get(pos); - } - - public int getSize() { - return columns.size(); - } - - public void clear() { - columns.clear(); - } - - - public TTableSchema toTTableSchema() { - TTableSchema tTableSchema = new TTableSchema(); - for (ColumnDescriptor col : columns) { - tTableSchema.addToColumns(col.toTColumnDesc()); - } - return tTableSchema; - } - - public Type[] toTypes() { - Type[] types = new Type[columns.size()]; - for (int i = 0; i < types.length; i++) { - types[i] = columns.get(i).getType(); - } - return types; - } - - public TableSchema addPrimitiveColumn(String columnName, Type columnType, String columnComment) { - columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, columnType, columns.size() + 1)); - return this; - } - - public TableSchema addStringColumn(String columnName, String columnComment) { - columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, Type.STRING_TYPE, columns.size() + 1)); - return this; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java deleted file mode 100644 index 7752ec03a29b7..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java +++ /dev/null @@ -1,349 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.sql.DatabaseMetaData; -import java.util.Locale; - -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hive.service.cli.thrift.TTypeId; - -/** - * Type. - * - */ -public enum Type { - NULL_TYPE("VOID", - java.sql.Types.NULL, - TTypeId.NULL_TYPE), - BOOLEAN_TYPE("BOOLEAN", - java.sql.Types.BOOLEAN, - TTypeId.BOOLEAN_TYPE), - TINYINT_TYPE("TINYINT", - java.sql.Types.TINYINT, - TTypeId.TINYINT_TYPE), - SMALLINT_TYPE("SMALLINT", - java.sql.Types.SMALLINT, - TTypeId.SMALLINT_TYPE), - INT_TYPE("INT", - java.sql.Types.INTEGER, - TTypeId.INT_TYPE), - BIGINT_TYPE("BIGINT", - java.sql.Types.BIGINT, - TTypeId.BIGINT_TYPE), - FLOAT_TYPE("FLOAT", - java.sql.Types.FLOAT, - TTypeId.FLOAT_TYPE), - DOUBLE_TYPE("DOUBLE", - java.sql.Types.DOUBLE, - TTypeId.DOUBLE_TYPE), - STRING_TYPE("STRING", - java.sql.Types.VARCHAR, - TTypeId.STRING_TYPE), - CHAR_TYPE("CHAR", - java.sql.Types.CHAR, - TTypeId.CHAR_TYPE, - true, false, false), - VARCHAR_TYPE("VARCHAR", - java.sql.Types.VARCHAR, - TTypeId.VARCHAR_TYPE, - true, false, false), - DATE_TYPE("DATE", - java.sql.Types.DATE, - TTypeId.DATE_TYPE), - TIMESTAMP_TYPE("TIMESTAMP", - java.sql.Types.TIMESTAMP, - TTypeId.TIMESTAMP_TYPE), - INTERVAL_YEAR_MONTH_TYPE("INTERVAL_YEAR_MONTH", - java.sql.Types.OTHER, - TTypeId.INTERVAL_YEAR_MONTH_TYPE), - INTERVAL_DAY_TIME_TYPE("INTERVAL_DAY_TIME", - java.sql.Types.OTHER, - TTypeId.INTERVAL_DAY_TIME_TYPE), - BINARY_TYPE("BINARY", - java.sql.Types.BINARY, - TTypeId.BINARY_TYPE), - DECIMAL_TYPE("DECIMAL", - java.sql.Types.DECIMAL, - TTypeId.DECIMAL_TYPE, - true, false, false), - ARRAY_TYPE("ARRAY", - java.sql.Types.ARRAY, - TTypeId.ARRAY_TYPE, - true, true), - MAP_TYPE("MAP", - java.sql.Types.JAVA_OBJECT, - TTypeId.MAP_TYPE, - true, true), - STRUCT_TYPE("STRUCT", - java.sql.Types.STRUCT, - TTypeId.STRUCT_TYPE, - true, false), - UNION_TYPE("UNIONTYPE", - java.sql.Types.OTHER, - TTypeId.UNION_TYPE, - true, false), - USER_DEFINED_TYPE("USER_DEFINED", - java.sql.Types.OTHER, - TTypeId.USER_DEFINED_TYPE, - true, false); - - private final String name; - private final TTypeId tType; - private final int javaSQLType; - private final boolean isQualified; - private final boolean isComplex; - private final boolean isCollection; - - Type(String name, int javaSQLType, TTypeId tType, boolean isQualified, boolean isComplex, boolean isCollection) { - this.name = name; - this.javaSQLType = javaSQLType; - this.tType = tType; - this.isQualified = isQualified; - this.isComplex = isComplex; - this.isCollection = isCollection; - } - - Type(String name, int javaSQLType, TTypeId tType, boolean isComplex, boolean isCollection) { - this(name, javaSQLType, tType, false, isComplex, isCollection); - } - - Type(String name, int javaSqlType, TTypeId tType) { - this(name, javaSqlType, tType, false, false, false); - } - - public boolean isPrimitiveType() { - return !isComplex; - } - - public boolean isQualifiedType() { - return isQualified; - } - - public boolean isComplexType() { - return isComplex; - } - - public boolean isCollectionType() { - return isCollection; - } - - public static Type getType(TTypeId tType) { - for (Type type : values()) { - if (tType.equals(type.tType)) { - return type; - } - } - throw new IllegalArgumentException("Unregonized Thrift TTypeId value: " + tType); - } - - public static Type getType(String name) { - if (name == null) { - throw new IllegalArgumentException("Invalid type name: null"); - } - for (Type type : values()) { - if (name.equalsIgnoreCase(type.name)) { - return type; - } else if (type.isQualifiedType() || type.isComplexType()) { - if (name.toUpperCase(Locale.ROOT).startsWith(type.name)) { - return type; - } - } - } - throw new IllegalArgumentException("Unrecognized type name: " + name); - } - - /** - * Radix for this type (typically either 2 or 10) - * Null is returned for data types where this is not applicable. - */ - public Integer getNumPrecRadix() { - if (this.isNumericType()) { - return 10; - } - return null; - } - - /** - * Maximum precision for numeric types. - * Returns null for non-numeric types. - * @return - */ - public Integer getMaxPrecision() { - switch (this) { - case TINYINT_TYPE: - return 3; - case SMALLINT_TYPE: - return 5; - case INT_TYPE: - return 10; - case BIGINT_TYPE: - return 19; - case FLOAT_TYPE: - return 7; - case DOUBLE_TYPE: - return 15; - case DECIMAL_TYPE: - return HiveDecimal.MAX_PRECISION; - default: - return null; - } - } - - public boolean isNumericType() { - switch (this) { - case TINYINT_TYPE: - case SMALLINT_TYPE: - case INT_TYPE: - case BIGINT_TYPE: - case FLOAT_TYPE: - case DOUBLE_TYPE: - case DECIMAL_TYPE: - return true; - default: - return false; - } - } - - /** - * Prefix used to quote a literal of this type (may be null) - */ - public String getLiteralPrefix() { - return null; - } - - /** - * Suffix used to quote a literal of this type (may be null) - * @return - */ - public String getLiteralSuffix() { - return null; - } - - /** - * Can you use NULL for this type? - * @return - * DatabaseMetaData.typeNoNulls - does not allow NULL values - * DatabaseMetaData.typeNullable - allows NULL values - * DatabaseMetaData.typeNullableUnknown - nullability unknown - */ - public Short getNullable() { - // All Hive types are nullable - return DatabaseMetaData.typeNullable; - } - - /** - * Is the type case sensitive? - * @return - */ - public Boolean isCaseSensitive() { - switch (this) { - case STRING_TYPE: - return true; - default: - return false; - } - } - - /** - * Parameters used in creating the type (may be null) - * @return - */ - public String getCreateParams() { - return null; - } - - /** - * Can you use WHERE based on this type? - * @return - * DatabaseMetaData.typePredNone - No support - * DatabaseMetaData.typePredChar - Only support with WHERE .. LIKE - * DatabaseMetaData.typePredBasic - Supported except for WHERE .. LIKE - * DatabaseMetaData.typeSearchable - Supported for all WHERE .. - */ - public Short getSearchable() { - if (isPrimitiveType()) { - return DatabaseMetaData.typeSearchable; - } - return DatabaseMetaData.typePredNone; - } - - /** - * Is this type unsigned? - * @return - */ - public Boolean isUnsignedAttribute() { - if (isNumericType()) { - return false; - } - return true; - } - - /** - * Can this type represent money? - * @return - */ - public Boolean isFixedPrecScale() { - return false; - } - - /** - * Can this type be used for an auto-increment value? - * @return - */ - public Boolean isAutoIncrement() { - return false; - } - - /** - * Localized version of type name (may be null). - * @return - */ - public String getLocalizedName() { - return null; - } - - /** - * Minimum scale supported for this type - * @return - */ - public Short getMinimumScale() { - return 0; - } - - /** - * Maximum scale supported for this type - * @return - */ - public Short getMaximumScale() { - return 0; - } - - public TTypeId toTType() { - return tType; - } - - public int toJavaSQLType() { - return javaSQLType; - } - - public String getName() { - return name; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java deleted file mode 100644 index b80fd67884add..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java +++ /dev/null @@ -1,159 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.List; - -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hive.service.cli.thrift.TPrimitiveTypeEntry; -import org.apache.hive.service.cli.thrift.TTypeDesc; -import org.apache.hive.service.cli.thrift.TTypeEntry; - -/** - * TypeDescriptor. - * - */ -public class TypeDescriptor { - - private final Type type; - private String typeName = null; - private TypeQualifiers typeQualifiers = null; - - public TypeDescriptor(Type type) { - this.type = type; - } - - public TypeDescriptor(TTypeDesc tTypeDesc) { - List tTypeEntries = tTypeDesc.getTypes(); - TPrimitiveTypeEntry top = tTypeEntries.get(0).getPrimitiveEntry(); - this.type = Type.getType(top.getType()); - if (top.isSetTypeQualifiers()) { - setTypeQualifiers(TypeQualifiers.fromTTypeQualifiers(top.getTypeQualifiers())); - } - } - - public TypeDescriptor(String typeName) { - this.type = Type.getType(typeName); - if (this.type.isComplexType()) { - this.typeName = typeName; - } else if (this.type.isQualifiedType()) { - PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(typeName); - setTypeQualifiers(TypeQualifiers.fromTypeInfo(pti)); - } - } - - public Type getType() { - return type; - } - - public TTypeDesc toTTypeDesc() { - TPrimitiveTypeEntry primitiveEntry = new TPrimitiveTypeEntry(type.toTType()); - if (getTypeQualifiers() != null) { - primitiveEntry.setTypeQualifiers(getTypeQualifiers().toTTypeQualifiers()); - } - TTypeEntry entry = TTypeEntry.primitiveEntry(primitiveEntry); - - TTypeDesc desc = new TTypeDesc(); - desc.addToTypes(entry); - return desc; - } - - public String getTypeName() { - if (typeName != null) { - return typeName; - } else { - return type.getName(); - } - } - - public TypeQualifiers getTypeQualifiers() { - return typeQualifiers; - } - - public void setTypeQualifiers(TypeQualifiers typeQualifiers) { - this.typeQualifiers = typeQualifiers; - } - - /** - * The column size for this type. - * For numeric data this is the maximum precision. - * For character data this is the length in characters. - * For datetime types this is the length in characters of the String representation - * (assuming the maximum allowed precision of the fractional seconds component). - * For binary data this is the length in bytes. - * Null is returned for data types where the column size is not applicable. - */ - public Integer getColumnSize() { - if (type.isNumericType()) { - return getPrecision(); - } - switch (type) { - case STRING_TYPE: - case BINARY_TYPE: - return Integer.MAX_VALUE; - case CHAR_TYPE: - case VARCHAR_TYPE: - return typeQualifiers.getCharacterMaximumLength(); - case DATE_TYPE: - return 10; - case TIMESTAMP_TYPE: - return 29; - default: - return null; - } - } - - /** - * Maximum precision for numeric types. - * Returns null for non-numeric types. - * @return - */ - public Integer getPrecision() { - if (this.type == Type.DECIMAL_TYPE) { - return typeQualifiers.getPrecision(); - } - return this.type.getMaxPrecision(); - } - - /** - * The number of fractional digits for this type. - * Null is returned for data types where this is not applicable. - */ - public Integer getDecimalDigits() { - switch (this.type) { - case BOOLEAN_TYPE: - case TINYINT_TYPE: - case SMALLINT_TYPE: - case INT_TYPE: - case BIGINT_TYPE: - return 0; - case FLOAT_TYPE: - return 7; - case DOUBLE_TYPE: - return 15; - case DECIMAL_TYPE: - return typeQualifiers.getScale(); - case TIMESTAMP_TYPE: - return 9; - default: - return null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java deleted file mode 100644 index c6da52c15a2b5..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * 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.hive.service.cli; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; -import org.apache.hive.service.cli.thrift.TCLIServiceConstants; -import org.apache.hive.service.cli.thrift.TTypeQualifierValue; -import org.apache.hive.service.cli.thrift.TTypeQualifiers; - -/** - * This class holds type qualifier information for a primitive type, - * such as char/varchar length or decimal precision/scale. - */ -public class TypeQualifiers { - private Integer characterMaximumLength; - private Integer precision; - private Integer scale; - - public TypeQualifiers() {} - - public Integer getCharacterMaximumLength() { - return characterMaximumLength; - } - public void setCharacterMaximumLength(int characterMaximumLength) { - this.characterMaximumLength = characterMaximumLength; - } - - public TTypeQualifiers toTTypeQualifiers() { - TTypeQualifiers ret = null; - - Map qMap = new HashMap(); - if (getCharacterMaximumLength() != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(getCharacterMaximumLength().intValue()); - qMap.put(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH, val); - } - - if (precision != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(precision.intValue()); - qMap.put(TCLIServiceConstants.PRECISION, val); - } - - if (scale != null) { - TTypeQualifierValue val = new TTypeQualifierValue(); - val.setI32Value(scale.intValue()); - qMap.put(TCLIServiceConstants.SCALE, val); - } - - if (qMap.size() > 0) { - ret = new TTypeQualifiers(qMap); - } - - return ret; - } - - public static TypeQualifiers fromTTypeQualifiers(TTypeQualifiers ttq) { - TypeQualifiers ret = null; - if (ttq != null) { - ret = new TypeQualifiers(); - Map tqMap = ttq.getQualifiers(); - - if (tqMap.containsKey(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH)) { - ret.setCharacterMaximumLength( - tqMap.get(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH).getI32Value()); - } - - if (tqMap.containsKey(TCLIServiceConstants.PRECISION)) { - ret.setPrecision(tqMap.get(TCLIServiceConstants.PRECISION).getI32Value()); - } - - if (tqMap.containsKey(TCLIServiceConstants.SCALE)) { - ret.setScale(tqMap.get(TCLIServiceConstants.SCALE).getI32Value()); - } - } - return ret; - } - - public static TypeQualifiers fromTypeInfo(PrimitiveTypeInfo pti) { - TypeQualifiers result = null; - if (pti instanceof VarcharTypeInfo) { - result = new TypeQualifiers(); - result.setCharacterMaximumLength(((VarcharTypeInfo)pti).getLength()); - } else if (pti instanceof CharTypeInfo) { - result = new TypeQualifiers(); - result.setCharacterMaximumLength(((CharTypeInfo)pti).getLength()); - } else if (pti instanceof DecimalTypeInfo) { - result = new TypeQualifiers(); - result.setPrecision(((DecimalTypeInfo)pti).precision()); - result.setScale(((DecimalTypeInfo)pti).scale()); - } - return result; - } - - public Integer getPrecision() { - return precision; - } - - public void setPrecision(Integer precision) { - this.precision = precision; - } - - public Integer getScale() { - return scale; - } - - public void setScale(Integer scale) { - this.scale = scale; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java deleted file mode 100644 index af36057bdaeca..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ /dev/null @@ -1,86 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.TableType; - -/** - * ClassicTableTypeMapping. - * Classic table type mapping : - * Managed Table to Table - * External Table to Table - * Virtual View to View - */ -public class ClassicTableTypeMapping implements TableTypeMapping { - - public enum ClassicTableTypes { - TABLE, - VIEW, - } - - private final Map hiveToClientMap = new HashMap(); - private final Map clientToHiveMap = new HashMap(); - - public ClassicTableTypeMapping() { - hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), - ClassicTableTypes.TABLE.toString()); - hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), - ClassicTableTypes.TABLE.toString()); - hiveToClientMap.put(TableType.VIRTUAL_VIEW.toString(), - ClassicTableTypes.VIEW.toString()); - - clientToHiveMap.put(ClassicTableTypes.TABLE.toString(), - TableType.MANAGED_TABLE.toString()); - clientToHiveMap.put(ClassicTableTypes.VIEW.toString(), - TableType.VIRTUAL_VIEW.toString()); - } - - @Override - public String mapToHiveType(String clientTypeName) { - if (clientToHiveMap.containsKey(clientTypeName)) { - return clientToHiveMap.get(clientTypeName); - } else { - return clientTypeName; - } - } - - @Override - public String mapToClientType(String hiveTypeName) { - if (hiveToClientMap.containsKey(hiveTypeName)) { - return hiveToClientMap.get(hiveTypeName); - } else { - return hiveTypeName; - } - } - - @Override - public Set getTableTypeNames() { - Set typeNameSet = new HashSet(); - for (ClassicTableTypes typeNames : ClassicTableTypes.values()) { - typeNameSet.add(typeNames.toString()); - } - return typeNameSet; - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java deleted file mode 100644 index 6740d3bb59dc3..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java +++ /dev/null @@ -1,83 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.hive.ql.processors.CommandProcessor; -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.session.HiveSession; - -public abstract class ExecuteStatementOperation extends Operation { - protected String statement = null; - protected Map confOverlay = new HashMap(); - - public ExecuteStatementOperation(HiveSession parentSession, String statement, - Map confOverlay, boolean runInBackground) { - super(parentSession, OperationType.EXECUTE_STATEMENT, runInBackground); - this.statement = statement; - setConfOverlay(confOverlay); - } - - public String getStatement() { - return statement; - } - - public static ExecuteStatementOperation newExecuteStatementOperation( - HiveSession parentSession, String statement, Map confOverlay, boolean runAsync) - throws HiveSQLException { - String[] tokens = statement.trim().split("\\s+"); - CommandProcessor processor = null; - try { - processor = CommandProcessorFactory.getForHiveCommand(tokens, parentSession.getHiveConf()); - } catch (SQLException e) { - throw new HiveSQLException(e.getMessage(), e.getSQLState(), e); - } - if (processor == null) { - return new SQLOperation(parentSession, statement, confOverlay, runAsync); - } - return new HiveCommandOperation(parentSession, statement, processor, confOverlay); - } - - protected Map getConfOverlay() { - return confOverlay; - } - - protected void setConfOverlay(Map confOverlay) { - if (confOverlay != null) { - this.confOverlay = confOverlay; - } - } - - protected void registerCurrentOperationLog() { - if (isOperationLogEnabled) { - if (operationLog == null) { - LOG.warn("Failed to get current OperationLog object of Operation: " + - getHandle().getHandleIdentifier()); - isOperationLogEnabled = false; - return; - } - OperationLog.setCurrentOperationLog(operationLog); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java deleted file mode 100644 index 581d975344060..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetCatalogsOperation. - * - */ -public class GetCatalogsOperation extends MetadataOperation { - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable."); - - protected final RowSet rowSet; - - protected GetCatalogsOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_CATALOGS); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_CATALOGS, null); - } - setState(OperationState.FINISHED); - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } - - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java deleted file mode 100644 index 96ba4890075ac..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ /dev/null @@ -1,234 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.sql.DatabaseMetaData; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.regex.Pattern; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; -import org.apache.hive.service.cli.ColumnDescriptor; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetColumnsOperation. - * - */ -public class GetColumnsOperation extends MetadataOperation { - - protected static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, - "Catalog name. NULL if not applicable") - .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, - "Schema name") - .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, - "Table name") - .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, - "Column name") - .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, - "SQL type from java.sql.Types") - .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, - "Data source dependent type name, for a UDT the type name is fully qualified") - .addPrimitiveColumn("COLUMN_SIZE", Type.INT_TYPE, - "Column size. For char or date types this is the maximum number of characters," - + " for numeric or decimal types this is precision.") - .addPrimitiveColumn("BUFFER_LENGTH", Type.TINYINT_TYPE, - "Unused") - .addPrimitiveColumn("DECIMAL_DIGITS", Type.INT_TYPE, - "The number of fractional digits") - .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, - "Radix (typically either 10 or 2)") - .addPrimitiveColumn("NULLABLE", Type.INT_TYPE, - "Is NULL allowed") - .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, - "Comment describing column (may be null)") - .addPrimitiveColumn("COLUMN_DEF", Type.STRING_TYPE, - "Default value (may be null)") - .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("CHAR_OCTET_LENGTH", Type.INT_TYPE, - "For char types the maximum number of bytes in the column") - .addPrimitiveColumn("ORDINAL_POSITION", Type.INT_TYPE, - "Index of column in table (starting at 1)") - .addPrimitiveColumn("IS_NULLABLE", Type.STRING_TYPE, - "\"NO\" means column definitely does not allow NULL values; " - + "\"YES\" means the column might allow NULL values. An empty " - + "string means nobody knows.") - .addPrimitiveColumn("SCOPE_CATALOG", Type.STRING_TYPE, - "Catalog of table that is the scope of a reference attribute " - + "(null if DATA_TYPE isn't REF)") - .addPrimitiveColumn("SCOPE_SCHEMA", Type.STRING_TYPE, - "Schema of table that is the scope of a reference attribute " - + "(null if the DATA_TYPE isn't REF)") - .addPrimitiveColumn("SCOPE_TABLE", Type.STRING_TYPE, - "Table name that this the scope of a reference attribure " - + "(null if the DATA_TYPE isn't REF)") - .addPrimitiveColumn("SOURCE_DATA_TYPE", Type.SMALLINT_TYPE, - "Source type of a distinct type or user-generated Ref type, " - + "SQL type from java.sql.Types (null if DATA_TYPE isn't DISTINCT or user-generated REF)") - .addPrimitiveColumn("IS_AUTO_INCREMENT", Type.STRING_TYPE, - "Indicates whether this column is auto incremented."); - - private final String catalogName; - private final String schemaName; - private final String tableName; - private final String columnName; - - protected final RowSet rowSet; - - protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, - String tableName, String columnName) { - super(parentSession, OperationType.GET_COLUMNS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.tableName = tableName; - this.columnName = columnName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - String tablePattern = convertIdentifierPattern(tableName, true); - - Pattern columnPattern = null; - if (columnName != null) { - columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); - } - - List dbNames = metastoreClient.getDatabases(schemaPattern); - Collections.sort(dbNames); - Map> db2Tabs = new HashMap<>(); - - for (String dbName : dbNames) { - List tableNames = metastoreClient.getTables(dbName, tablePattern); - Collections.sort(tableNames); - db2Tabs.put(dbName, tableNames); - } - - if (isAuthV2Enabled()) { - List privObjs = getPrivObjs(db2Tabs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName - + ", tablePattern : " + tableName; - authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr); - } - - for (Entry> dbTabs : db2Tabs.entrySet()) { - String dbName = dbTabs.getKey(); - List tableNames = dbTabs.getValue(); - for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { - TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, table.getTableName())); - for (ColumnDescriptor column : schema.getColumnDescriptors()) { - if (columnPattern != null && !columnPattern.matcher(column.getName()).matches()) { - continue; - } - Object[] rowData = new Object[] { - null, // TABLE_CAT - table.getDbName(), // TABLE_SCHEM - table.getTableName(), // TABLE_NAME - column.getName(), // COLUMN_NAME - column.getType().toJavaSQLType(), // DATA_TYPE - column.getTypeName(), // TYPE_NAME - column.getTypeDescriptor().getColumnSize(), // COLUMN_SIZE - null, // BUFFER_LENGTH, unused - column.getTypeDescriptor().getDecimalDigits(), // DECIMAL_DIGITS - column.getType().getNumPrecRadix(), // NUM_PREC_RADIX - DatabaseMetaData.columnNullable, // NULLABLE - column.getComment(), // REMARKS - null, // COLUMN_DEF - null, // SQL_DATA_TYPE - null, // SQL_DATETIME_SUB - null, // CHAR_OCTET_LENGTH - column.getOrdinalPosition(), // ORDINAL_POSITION - "YES", // IS_NULLABLE - null, // SCOPE_CATALOG - null, // SCOPE_SCHEMA - null, // SCOPE_TABLE - null, // SOURCE_DATA_TYPE - "NO", // IS_AUTO_INCREMENT - }; - rowSet.addRow(rowData); - } - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - - } - - - private List getPrivObjs(Map> db2Tabs) { - List privObjs = new ArrayList<>(); - for (Entry> dbTabs : db2Tabs.entrySet()) { - for (String tabName : dbTabs.getValue()) { - privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(), - tabName)); - } - } - return privObjs; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java deleted file mode 100644 index 5dec8bdbf45de..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ /dev/null @@ -1,147 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.sql.DatabaseMetaData; -import java.util.List; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.exec.FunctionInfo; -import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; -import org.apache.hive.service.cli.CLIServiceUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.thrift.TException; - -/** - * GetFunctionsOperation. - * - */ -public class GetFunctionsOperation extends MetadataOperation { - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("FUNCTION_CAT", Type.STRING_TYPE, - "Function catalog (may be null)") - .addPrimitiveColumn("FUNCTION_SCHEM", Type.STRING_TYPE, - "Function schema (may be null)") - .addPrimitiveColumn("FUNCTION_NAME", Type.STRING_TYPE, - "Function name. This is the name used to invoke the function") - .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, - "Explanatory comment on the function") - .addPrimitiveColumn("FUNCTION_TYPE", Type.INT_TYPE, - "Kind of function.") - .addPrimitiveColumn("SPECIFIC_NAME", Type.STRING_TYPE, - "The name which uniquely identifies this function within its schema"); - - private final String catalogName; - private final String schemaName; - private final String functionName; - - protected final RowSet rowSet; - - public GetFunctionsOperation(HiveSession parentSession, - String catalogName, String schemaName, String functionName) { - super(parentSession, OperationType.GET_FUNCTIONS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.functionName = functionName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - // get databases for schema pattern - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - List matchingDbs; - try { - matchingDbs = metastoreClient.getDatabases(schemaPattern); - } catch (TException e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - // authorize this call on the schema objects - List privObjs = HivePrivilegeObjectUtils - .getHivePrivDbObjects(matchingDbs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr); - } - - try { - if ((null == catalogName || "".equals(catalogName)) - && (null == schemaName || "".equals(schemaName))) { - Set functionNames = FunctionRegistry - .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); - for (String functionName : functionNames) { - FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); - Object[] rowData = new Object[] { - null, // FUNCTION_CAT - null, // FUNCTION_SCHEM - functionInfo.getDisplayName(), // FUNCTION_NAME - "", // REMARKS - (functionInfo.isGenericUDTF() ? - DatabaseMetaData.functionReturnsTable - : DatabaseMetaData.functionNoTable), // FUNCTION_TYPE - functionInfo.getClass().getCanonicalName() - }; - rowSet.addRow(rowData); - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java deleted file mode 100644 index 3516bc2ba242c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetSchemasOperation. - * - */ -public class GetSchemasOperation extends MetadataOperation { - private final String catalogName; - private final String schemaName; - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_SCHEM", "Schema name.") - .addStringColumn("TABLE_CATALOG", "Catalog name."); - - protected RowSet rowSet; - - protected GetSchemasOperation(HiveSession parentSession, - String catalogName, String schemaName) { - super(parentSession, OperationType.GET_SCHEMAS); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr); - } - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - for (String dbName : metastoreClient.getDatabases(schemaPattern)) { - rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java deleted file mode 100644 index b372f55cedd1c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ /dev/null @@ -1,93 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTableTypesOperation. - * - */ -public class GetTableTypesOperation extends MetadataOperation { - - protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_TYPE", "Table type name."); - - protected final RowSet rowSet; - private final TableTypeMapping tableTypeMapping; - - protected GetTableTypesOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TABLE_TYPES); - String tableMappingStr = getParentSession().getHiveConf() - .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); - tableTypeMapping = - TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null); - } - try { - for (TableType type : TableType.values()) { - rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())}); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java deleted file mode 100644 index 2af17a662a296..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ /dev/null @@ -1,135 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTablesOperation. - * - */ -public class GetTablesOperation extends MetadataOperation { - - private final String catalogName; - private final String schemaName; - private final String tableName; - private final List tableTypes = new ArrayList(); - protected final RowSet rowSet; - private final TableTypeMapping tableTypeMapping; - - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable.") - .addStringColumn("TABLE_SCHEM", "Schema name.") - .addStringColumn("TABLE_NAME", "Table name.") - .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.") - .addStringColumn("REMARKS", "Comments about the table."); - - protected GetTablesOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, - List tableTypes) { - super(parentSession, OperationType.GET_TABLES); - this.catalogName = catalogName; - this.schemaName = schemaName; - this.tableName = tableName; - String tableMappingStr = getParentSession().getHiveConf() - .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); - tableTypeMapping = - TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); - if (tableTypes != null) { - this.tableTypes.addAll(tableTypes); - } - this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); - String schemaPattern = convertSchemaPattern(schemaName); - List matchingDbs = metastoreClient.getDatabases(schemaPattern); - if(isAuthV2Enabled()){ - List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); - String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; - authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); - } - - String tablePattern = convertIdentifierPattern(tableName, true); - for (String dbName : metastoreClient.getDatabases(schemaPattern)) { - List tableNames = metastoreClient.getTables(dbName, tablePattern); - for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { - Object[] rowData = new Object[] { - DEFAULT_HIVE_CATALOG, - table.getDbName(), - table.getTableName(), - tableTypeMapping.mapToClientType(table.getTableType()), - table.getParameters().get("comment") - }; - if (tableTypes.isEmpty() || tableTypes.contains( - tableTypeMapping.mapToClientType(table.getTableType()))) { - rowSet.addRow(rowData); - } - } - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java deleted file mode 100644 index 3e81f8afbd85f..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ /dev/null @@ -1,142 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.Type; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * GetTypeInfoOperation. - * - */ -public class GetTypeInfoOperation extends MetadataOperation { - - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() - .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, - "Type name") - .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, - "SQL data type from java.sql.Types") - .addPrimitiveColumn("PRECISION", Type.INT_TYPE, - "Maximum precision") - .addPrimitiveColumn("LITERAL_PREFIX", Type.STRING_TYPE, - "Prefix used to quote a literal (may be null)") - .addPrimitiveColumn("LITERAL_SUFFIX", Type.STRING_TYPE, - "Suffix used to quote a literal (may be null)") - .addPrimitiveColumn("CREATE_PARAMS", Type.STRING_TYPE, - "Parameters used in creating the type (may be null)") - .addPrimitiveColumn("NULLABLE", Type.SMALLINT_TYPE, - "Can you use NULL for this type") - .addPrimitiveColumn("CASE_SENSITIVE", Type.BOOLEAN_TYPE, - "Is it case sensitive") - .addPrimitiveColumn("SEARCHABLE", Type.SMALLINT_TYPE, - "Can you use \"WHERE\" based on this type") - .addPrimitiveColumn("UNSIGNED_ATTRIBUTE", Type.BOOLEAN_TYPE, - "Is it unsigned") - .addPrimitiveColumn("FIXED_PREC_SCALE", Type.BOOLEAN_TYPE, - "Can it be a money value") - .addPrimitiveColumn("AUTO_INCREMENT", Type.BOOLEAN_TYPE, - "Can it be used for an auto-increment value") - .addPrimitiveColumn("LOCAL_TYPE_NAME", Type.STRING_TYPE, - "Localized version of type name (may be null)") - .addPrimitiveColumn("MINIMUM_SCALE", Type.SMALLINT_TYPE, - "Minimum scale supported") - .addPrimitiveColumn("MAXIMUM_SCALE", Type.SMALLINT_TYPE, - "Maximum scale supported") - .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, - "Unused") - .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, - "Usually 2 or 10"); - - protected final RowSet rowSet; - - protected GetTypeInfoOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TYPE_INFO); - rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - if (isAuthV2Enabled()) { - authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null); - } - try { - for (Type type : Type.values()) { - Object[] rowData = new Object[] { - type.getName(), // TYPE_NAME - type.toJavaSQLType(), // DATA_TYPE - type.getMaxPrecision(), // PRECISION - type.getLiteralPrefix(), // LITERAL_PREFIX - type.getLiteralSuffix(), // LITERAL_SUFFIX - type.getCreateParams(), // CREATE_PARAMS - type.getNullable(), // NULLABLE - type.isCaseSensitive(), // CASE_SENSITIVE - type.getSearchable(), // SEARCHABLE - type.isUnsignedAttribute(), // UNSIGNED_ATTRIBUTE - type.isFixedPrecScale(), // FIXED_PREC_SCALE - type.isAutoIncrement(), // AUTO_INCREMENT - type.getLocalizedName(), // LOCAL_TYPE_NAME - type.getMinimumScale(), // MINIMUM_SCALE - type.getMaximumScale(), // MAXIMUM_SCALE - null, // SQL_DATA_TYPE, unused - null, // SQL_DATETIME_SUB, unused - type.getNumPrecRadix() //NUM_PREC_RADIX - }; - rowSet.addRow(rowData); - } - setState(OperationState.FINISHED); - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException(e); - } - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - return RESULT_SET_SCHEMA; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - assertState(OperationState.FINISHED); - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - rowSet.setStartOffset(0); - } - return rowSet.extractSubset((int)maxRows); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java deleted file mode 100644 index 5b6e6ad042412..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java +++ /dev/null @@ -1,215 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import static java.nio.charset.StandardCharsets.UTF_8; - -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.processors.CommandProcessor; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.IOUtils; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * Executes a HiveCommand - */ -public class HiveCommandOperation extends ExecuteStatementOperation { - private CommandProcessor commandProcessor; - private TableSchema resultSchema = null; - - /** - * For processors other than Hive queries (Driver), they output to session.out (a temp file) - * first and the fetchOne/fetchN/fetchAll functions get the output from pipeIn. - */ - private BufferedReader resultReader; - - - protected HiveCommandOperation(HiveSession parentSession, String statement, - CommandProcessor commandProcessor, Map confOverlay) { - super(parentSession, statement, confOverlay, false); - this.commandProcessor = commandProcessor; - setupSessionIO(parentSession.getSessionState()); - } - - private void setupSessionIO(SessionState sessionState) { - try { - LOG.info("Putting temp output to file " + sessionState.getTmpOutputFile().toString()); - sessionState.in = null; // hive server's session input stream is not used - // open a per-session file in auto-flush mode for writing temp results - sessionState.out = new PrintStream(new FileOutputStream(sessionState.getTmpOutputFile()), true, UTF_8.name()); - // TODO: for hadoop jobs, progress is printed out to session.err, - // we should find a way to feed back job progress to client - sessionState.err = new PrintStream(System.err, true, UTF_8.name()); - } catch (IOException e) { - LOG.error("Error in creating temp output file ", e); - try { - sessionState.in = null; - sessionState.out = new PrintStream(System.out, true, UTF_8.name()); - sessionState.err = new PrintStream(System.err, true, UTF_8.name()); - } catch (UnsupportedEncodingException ee) { - LOG.error("Error creating PrintStream", e); - ee.printStackTrace(); - sessionState.out = null; - sessionState.err = null; - } - } - } - - - private void tearDownSessionIO() { - IOUtils.cleanup(LOG, parentSession.getSessionState().out); - IOUtils.cleanup(LOG, parentSession.getSessionState().err); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.RUNNING); - try { - String command = getStatement().trim(); - String[] tokens = statement.split("\\s"); - String commandArgs = command.substring(tokens[0].length()).trim(); - - CommandProcessorResponse response = commandProcessor.run(commandArgs); - int returnCode = response.getResponseCode(); - if (returnCode != 0) { - throw toSQLException("Error while processing statement", response); - } - Schema schema = response.getSchema(); - if (schema != null) { - setHasResultSet(true); - resultSchema = new TableSchema(schema); - } else { - setHasResultSet(false); - resultSchema = new TableSchema(); - } - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - setState(OperationState.FINISHED); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#close() - */ - @Override - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - tearDownSessionIO(); - cleanTmpFile(); - cleanupOperationLog(); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#getResultSetSchema() - */ - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - return resultSchema; - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.operation.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) - */ - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - validateDefaultFetchOrientation(orientation); - if (orientation.equals(FetchOrientation.FETCH_FIRST)) { - resetResultReader(); - } - List rows = readResults((int) maxRows); - RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); - - for (String row : rows) { - rowSet.addRow(new String[] {row}); - } - return rowSet; - } - - /** - * Reads the temporary results for non-Hive (non-Driver) commands to the - * resulting List of strings. - * @param nLines number of lines read at once. If it is <= 0, then read all lines. - */ - private List readResults(int nLines) throws HiveSQLException { - if (resultReader == null) { - SessionState sessionState = getParentSession().getSessionState(); - File tmp = sessionState.getTmpOutputFile(); - try { - resultReader = new BufferedReader(new FileReader(tmp)); - } catch (FileNotFoundException e) { - LOG.error("File " + tmp + " not found. ", e); - throw new HiveSQLException(e); - } - } - List results = new ArrayList(); - - for (int i = 0; i < nLines || nLines <= 0; ++i) { - try { - String line = resultReader.readLine(); - if (line == null) { - // reached the end of the result file - break; - } else { - results.add(line); - } - } catch (IOException e) { - LOG.error("Reading temp results encountered an exception: ", e); - throw new HiveSQLException(e); - } - } - return results; - } - - private void cleanTmpFile() { - resetResultReader(); - SessionState sessionState = getParentSession().getSessionState(); - File tmp = sessionState.getTmpOutputFile(); - tmp.delete(); - } - - private void resetResultReader() { - if (resultReader != null) { - IOUtils.cleanup(LOG, resultReader); - resultReader = null; - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java deleted file mode 100644 index b530f217125b8..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java +++ /dev/null @@ -1,51 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.hadoop.hive.metastore.TableType; - -/** - * HiveTableTypeMapping. - * Default table type mapping - * - */ -public class HiveTableTypeMapping implements TableTypeMapping { - - @Override - public String mapToHiveType(String clientTypeName) { - return clientTypeName; - } - - @Override - public String mapToClientType(String hiveTypeName) { - return hiveTypeName; - } - - @Override - public Set getTableTypeNames() { - Set typeNameSet = new HashSet(); - for (TableType typeNames : TableType.values()) { - typeNameSet.add(typeNames.toString()); - } - return typeNameSet; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java deleted file mode 100644 index 6c819876a556d..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ /dev/null @@ -1,134 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; - -/** - * MetadataOperation. - * - */ -public abstract class MetadataOperation extends Operation { - - protected static final String DEFAULT_HIVE_CATALOG = ""; - protected static TableSchema RESULT_SET_SCHEMA; - private static final char SEARCH_STRING_ESCAPE = '\\'; - - protected MetadataOperation(HiveSession parentSession, OperationType opType) { - super(parentSession, opType, false); - setHasResultSet(true); - } - - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.Operation#close() - */ - @Override - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - cleanupOperationLog(); - } - - /** - * Convert wildchars and escape sequence from JDBC format to datanucleous/regex - */ - protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { - if (pattern == null) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, datanucleusFormat); - } - } - - /** - * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex - * The schema pattern treats empty string also as wildchar - */ - protected String convertSchemaPattern(final String pattern) { - if ((pattern == null) || pattern.isEmpty()) { - return convertPattern("%", true); - } else { - return convertPattern(pattern, true); - } - } - - /** - * Convert a pattern containing JDBC catalog search wildcards into - * Java regex patterns. - * - * @param pattern input which may contain '%' or '_' wildcard characters, or - * these characters escaped using {@link #getSearchStringEscape()}. - * @return replace %/_ with regex search characters, also handle escaped - * characters. - * - * The datanucleus module expects the wildchar as '*'. The columns search on the - * other hand is done locally inside the hive code and that requires the regex wildchar - * format '.*' This is driven by the datanucleusFormat flag. - */ - private String convertPattern(final String pattern, boolean datanucleusFormat) { - String wStr; - if (datanucleusFormat) { - wStr = "*"; - } else { - wStr = ".*"; - } - return pattern - .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) - .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); - } - - protected boolean isAuthV2Enabled(){ - SessionState ss = SessionState.get(); - return (ss.isAuthorizationModeV2() && - HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)); - } - - protected void authorizeMetaGets(HiveOperationType opType, List inpObjs) - throws HiveSQLException { - authorizeMetaGets(opType, inpObjs, null); - } - - protected void authorizeMetaGets(HiveOperationType opType, List inpObjs, - String cmdString) throws HiveSQLException { - SessionState ss = SessionState.get(); - HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder(); - ctxBuilder.setUserIpAddress(ss.getUserIpAddress()); - ctxBuilder.setCommandString(cmdString); - try { - ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null, - ctxBuilder.build()); - } catch (HiveAuthzPluginException | HiveAccessControlException e) { - throw new HiveSQLException(e.getMessage(), e); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java deleted file mode 100644 index 4b331423948fa..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.io.File; -import java.io.FileNotFoundException; -import java.util.EnumSet; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.OperationType; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -public abstract class Operation { - protected final HiveSession parentSession; - private OperationState state = OperationState.INITIALIZED; - private final OperationHandle opHandle; - private HiveConf configuration; - public static final Log LOG = LogFactory.getLog(Operation.class.getName()); - public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; - public static final long DEFAULT_FETCH_MAX_ROWS = 100; - protected boolean hasResultSet; - protected volatile HiveSQLException operationException; - protected final boolean runAsync; - protected volatile Future backgroundHandle; - protected OperationLog operationLog; - protected boolean isOperationLogEnabled; - - private long operationTimeout; - private long lastAccessTime; - - protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of( - FetchOrientation.FETCH_NEXT, - FetchOrientation.FETCH_FIRST, - FetchOrientation.FETCH_PRIOR); - - protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { - this.parentSession = parentSession; - this.runAsync = runInBackground; - this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); - lastAccessTime = System.currentTimeMillis(); - operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), - HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); - } - - public Future getBackgroundHandle() { - return backgroundHandle; - } - - protected void setBackgroundHandle(Future backgroundHandle) { - this.backgroundHandle = backgroundHandle; - } - - public boolean shouldRunAsync() { - return runAsync; - } - - public void setConfiguration(HiveConf configuration) { - this.configuration = new HiveConf(configuration); - } - - public HiveConf getConfiguration() { - return new HiveConf(configuration); - } - - public HiveSession getParentSession() { - return parentSession; - } - - public OperationHandle getHandle() { - return opHandle; - } - - public TProtocolVersion getProtocolVersion() { - return opHandle.getProtocolVersion(); - } - - public OperationType getType() { - return opHandle.getOperationType(); - } - - public OperationStatus getStatus() { - return new OperationStatus(state, operationException); - } - - public boolean hasResultSet() { - return hasResultSet; - } - - protected void setHasResultSet(boolean hasResultSet) { - this.hasResultSet = hasResultSet; - opHandle.setHasResultSet(hasResultSet); - } - - public OperationLog getOperationLog() { - return operationLog; - } - - protected final OperationState setState(OperationState newState) throws HiveSQLException { - state.validateTransition(newState); - this.state = newState; - this.lastAccessTime = System.currentTimeMillis(); - return this.state; - } - - public boolean isTimedOut(long current) { - if (operationTimeout == 0) { - return false; - } - if (operationTimeout > 0) { - // check only when it's in terminal state - return state.isTerminal() && lastAccessTime + operationTimeout <= current; - } - return lastAccessTime + -operationTimeout <= current; - } - - public long getLastAccessTime() { - return lastAccessTime; - } - - public long getOperationTimeout() { - return operationTimeout; - } - - public void setOperationTimeout(long operationTimeout) { - this.operationTimeout = operationTimeout; - } - - protected void setOperationException(HiveSQLException operationException) { - this.operationException = operationException; - } - - protected final void assertState(OperationState state) throws HiveSQLException { - if (this.state != state) { - throw new HiveSQLException("Expected state " + state + ", but found " + this.state); - } - this.lastAccessTime = System.currentTimeMillis(); - } - - public boolean isRunning() { - return OperationState.RUNNING.equals(state); - } - - public boolean isFinished() { - return OperationState.FINISHED.equals(state); - } - - public boolean isCanceled() { - return OperationState.CANCELED.equals(state); - } - - public boolean isFailed() { - return OperationState.ERROR.equals(state); - } - - protected void createOperationLog() { - if (parentSession.isOperationLogEnabled()) { - File operationLogFile = new File(parentSession.getOperationLogSessionDir(), - opHandle.getHandleIdentifier().toString()); - isOperationLogEnabled = true; - - // create log file - try { - if (operationLogFile.exists()) { - LOG.warn("The operation log file should not exist, but it is already there: " + - operationLogFile.getAbsolutePath()); - operationLogFile.delete(); - } - if (!operationLogFile.createNewFile()) { - // the log file already exists and cannot be deleted. - // If it can be read/written, keep its contents and use it. - if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { - LOG.warn("The already existed operation log file cannot be recreated, " + - "and it cannot be read or written: " + operationLogFile.getAbsolutePath()); - isOperationLogEnabled = false; - return; - } - } - } catch (Exception e) { - LOG.warn("Unable to create operation log file: " + operationLogFile.getAbsolutePath(), e); - isOperationLogEnabled = false; - return; - } - - // create OperationLog object with above log file - try { - operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); - } catch (FileNotFoundException e) { - LOG.warn("Unable to instantiate OperationLog object for operation: " + - opHandle, e); - isOperationLogEnabled = false; - return; - } - - // register this operationLog to current thread - OperationLog.setCurrentOperationLog(operationLog); - } - } - - protected void unregisterOperationLog() { - if (isOperationLogEnabled) { - OperationLog.removeCurrentOperationLog(); - } - } - - /** - * Invoked before runInternal(). - * Set up some preconditions, or configurations. - */ - protected void beforeRun() { - createOperationLog(); - } - - /** - * Invoked after runInternal(), even if an exception is thrown in runInternal(). - * Clean up resources, which was set up in beforeRun(). - */ - protected void afterRun() { - unregisterOperationLog(); - } - - /** - * Implemented by subclass of Operation class to execute specific behaviors. - * @throws HiveSQLException - */ - protected abstract void runInternal() throws HiveSQLException; - - public void run() throws HiveSQLException { - beforeRun(); - try { - runInternal(); - } finally { - afterRun(); - } - } - - protected void cleanupOperationLog() { - if (isOperationLogEnabled) { - if (operationLog == null) { - LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] " - + "logging is enabled, but its OperationLog object cannot be found."); - } else { - operationLog.close(); - } - } - } - - // TODO: make this abstract and implement in subclasses. - public void cancel() throws HiveSQLException { - setState(OperationState.CANCELED); - throw new UnsupportedOperationException("SQLOperation.cancel()"); - } - - public void close() throws HiveSQLException { - setState(OperationState.CLOSED); - cleanupOperationLog(); - } - - public abstract TableSchema getResultSetSchema() throws HiveSQLException; - - public abstract RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException; - - public RowSet getNextRowSet() throws HiveSQLException { - return getNextRowSet(FetchOrientation.FETCH_NEXT, DEFAULT_FETCH_MAX_ROWS); - } - - /** - * Verify if the given fetch orientation is part of the default orientation types. - * @param orientation - * @throws HiveSQLException - */ - protected void validateDefaultFetchOrientation(FetchOrientation orientation) - throws HiveSQLException { - validateFetchOrientation(orientation, DEFAULT_FETCH_ORIENTATION_SET); - } - - /** - * Verify if the given fetch orientation is part of the supported orientation types. - * @param orientation - * @param supportedOrientations - * @throws HiveSQLException - */ - protected void validateFetchOrientation(FetchOrientation orientation, - EnumSet supportedOrientations) throws HiveSQLException { - if (!supportedOrientations.contains(orientation)) { - throw new HiveSQLException("The fetch type " + orientation.toString() + - " is not supported for this resultset", "HY106"); - } - } - - protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { - HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), - response.getSQLState(), response.getResponseCode()); - if (response.getException() != null) { - ex.initCause(response.getException()); - } - return ex; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java deleted file mode 100644 index 92c340a29c107..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java +++ /dev/null @@ -1,284 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.session.OperationLog; -import org.apache.hive.service.AbstractService; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.log4j.Appender; -import org.apache.log4j.Logger; - -/** - * OperationManager. - * - */ -public class OperationManager extends AbstractService { - private final Log LOG = LogFactory.getLog(OperationManager.class.getName()); - - private final Map handleToOperation = - new HashMap(); - - public OperationManager() { - super(OperationManager.class.getSimpleName()); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { - initOperationLogCapture(hiveConf.getVar( - HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); - } else { - LOG.debug("Operation level logging is turned off"); - } - super.init(hiveConf); - } - - @Override - public synchronized void start() { - super.start(); - // TODO - } - - @Override - public synchronized void stop() { - // TODO - super.stop(); - } - - private void initOperationLogCapture(String loggingMode) { - // Register another Appender (with the same layout) that talks to us. - Appender ap = new LogDivertAppender(this, OperationLog.getLoggingLevel(loggingMode)); - Logger.getRootLogger().addAppender(ap); - } - - public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, - String statement, Map confOverlay, boolean runAsync) - throws HiveSQLException { - ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation - .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync); - addOperation(executeStatementOperation); - return executeStatementOperation; - } - - public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { - GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { - GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, - String catalogName, String schemaName) { - GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); - addOperation(operation); - return operation; - } - - public MetadataOperation newGetTablesOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, - List tableTypes) { - MetadataOperation operation = - new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); - addOperation(operation); - return operation; - } - - public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { - GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); - addOperation(operation); - return operation; - } - - public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, - String catalogName, String schemaName, String tableName, String columnName) { - GetColumnsOperation operation = new GetColumnsOperation(parentSession, - catalogName, schemaName, tableName, columnName); - addOperation(operation); - return operation; - } - - public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, - String catalogName, String schemaName, String functionName) { - GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, - catalogName, schemaName, functionName); - addOperation(operation); - return operation; - } - - public Operation getOperation(OperationHandle operationHandle) throws HiveSQLException { - Operation operation = getOperationInternal(operationHandle); - if (operation == null) { - throw new HiveSQLException("Invalid OperationHandle: " + operationHandle); - } - return operation; - } - - private synchronized Operation getOperationInternal(OperationHandle operationHandle) { - return handleToOperation.get(operationHandle); - } - - private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) { - Operation operation = handleToOperation.get(operationHandle); - if (operation != null && operation.isTimedOut(System.currentTimeMillis())) { - handleToOperation.remove(operationHandle); - return operation; - } - return null; - } - - private synchronized void addOperation(Operation operation) { - handleToOperation.put(operation.getHandle(), operation); - } - - private synchronized Operation removeOperation(OperationHandle opHandle) { - return handleToOperation.remove(opHandle); - } - - public OperationStatus getOperationStatus(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getStatus(); - } - - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - Operation operation = getOperation(opHandle); - OperationState opState = operation.getStatus().getState(); - if (opState == OperationState.CANCELED || - opState == OperationState.CLOSED || - opState == OperationState.FINISHED || - opState == OperationState.ERROR || - opState == OperationState.UNKNOWN) { - // Cancel should be a no-op in either cases - LOG.debug(opHandle + ": Operation is already aborted in state - " + opState); - } - else { - LOG.debug(opHandle + ": Attempting to cancel from state - " + opState); - operation.cancel(); - } - } - - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - Operation operation = removeOperation(opHandle); - if (operation == null) { - throw new HiveSQLException("Operation does not exist!"); - } - operation.close(); - } - - public TableSchema getOperationResultSetSchema(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getResultSetSchema(); - } - - public RowSet getOperationNextRowSet(OperationHandle opHandle) - throws HiveSQLException { - return getOperation(opHandle).getNextRowSet(); - } - - public RowSet getOperationNextRowSet(OperationHandle opHandle, - FetchOrientation orientation, long maxRows) - throws HiveSQLException { - return getOperation(opHandle).getNextRowSet(orientation, maxRows); - } - - public RowSet getOperationLogRowSet(OperationHandle opHandle, - FetchOrientation orientation, long maxRows) - throws HiveSQLException { - // get the OperationLog object from the operation - OperationLog operationLog = getOperation(opHandle).getOperationLog(); - if (operationLog == null) { - throw new HiveSQLException("Couldn't find log associated with operation handle: " + opHandle); - } - - // read logs - List logs; - try { - logs = operationLog.readOperationLog(isFetchFirst(orientation), maxRows); - } catch (SQLException e) { - throw new HiveSQLException(e.getMessage(), e.getCause()); - } - - - // convert logs to RowSet - TableSchema tableSchema = new TableSchema(getLogSchema()); - RowSet rowSet = RowSetFactory.create(tableSchema, getOperation(opHandle).getProtocolVersion()); - for (String log : logs) { - rowSet.addRow(new String[] {log}); - } - - return rowSet; - } - - private boolean isFetchFirst(FetchOrientation fetchOrientation) { - //TODO: Since OperationLog is moved to package o.a.h.h.ql.session, - // we may add a Enum there and map FetchOrientation to it. - if (fetchOrientation.equals(FetchOrientation.FETCH_FIRST)) { - return true; - } - return false; - } - - private Schema getLogSchema() { - Schema schema = new Schema(); - FieldSchema fieldSchema = new FieldSchema(); - fieldSchema.setName("operation_log"); - fieldSchema.setType("string"); - schema.addToFieldSchemas(fieldSchema); - return schema; - } - - public OperationLog getOperationLogByThread() { - return OperationLog.getCurrentOperationLog(); - } - - public List removeExpiredOperations(OperationHandle[] handles) { - List removed = new ArrayList(); - for (OperationHandle handle : handles) { - Operation operation = removeTimedOutOperation(handle); - if (operation != null) { - LOG.warn("Operation " + handle + " is timed-out and will be closed"); - removed.add(operation); - } - } - return removed; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java deleted file mode 100644 index c7726f1fac07a..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ /dev/null @@ -1,456 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.io.IOException; -import java.io.Serializable; -import java.nio.charset.StandardCharsets; -import java.security.PrivilegedExceptionAction; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.Future; -import java.util.concurrent.RejectedExecutionException; - -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.CommandNeedRetryException; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.exec.ExplainTask; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.SerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationState; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.RowSetFactory; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.session.HiveSession; -import org.apache.hive.service.server.ThreadWithGarbageCleanup; - -/** - * SQLOperation. - * - */ -public class SQLOperation extends ExecuteStatementOperation { - - private Driver driver = null; - private CommandProcessorResponse response; - private TableSchema resultSchema = null; - private Schema mResultSchema = null; - private SerDe serde = null; - private boolean fetchStarted = false; - - public SQLOperation(HiveSession parentSession, String statement, Map confOverlay, boolean runInBackground) { - // TODO: call setRemoteUser in ExecuteStatementOperation or higher. - super(parentSession, statement, confOverlay, runInBackground); - } - - /*** - * Compile the query and extract metadata - * @param sqlOperationConf - * @throws HiveSQLException - */ - public void prepare(HiveConf sqlOperationConf) throws HiveSQLException { - setState(OperationState.RUNNING); - - try { - driver = new Driver(sqlOperationConf, getParentSession().getUserName()); - - // set the operation handle information in Driver, so that thrift API users - // can use the operation handle they receive, to lookup query information in - // Yarn ATS - String guid64 = Base64.encodeBase64URLSafeString(getHandle().getHandleIdentifier() - .toTHandleIdentifier().getGuid()).trim(); - driver.setOperationId(guid64); - - // In Hive server mode, we are not able to retry in the FetchTask - // case, when calling fetch queries since execute() has returned. - // For now, we disable the test attempts. - driver.setTryCount(Integer.MAX_VALUE); - - String subStatement = new VariableSubstitution().substitute(sqlOperationConf, statement); - response = driver.compileAndRespond(subStatement); - if (0 != response.getResponseCode()) { - throw toSQLException("Error while compiling statement", response); - } - - mResultSchema = driver.getSchema(); - - // hasResultSet should be true only if the query has a FetchTask - // "explain" is an exception for now - if(driver.getPlan().getFetchTask() != null) { - //Schema has to be set - if (mResultSchema == null || !mResultSchema.isSetFieldSchemas()) { - throw new HiveSQLException("Error compiling query: Schema and FieldSchema " + - "should be set when query plan has a FetchTask"); - } - resultSchema = new TableSchema(mResultSchema); - setHasResultSet(true); - } else { - setHasResultSet(false); - } - // Set hasResultSet true if the plan has ExplainTask - // TODO explain should use a FetchTask for reading - for (Task task: driver.getPlan().getRootTasks()) { - if (task.getClass() == ExplainTask.class) { - resultSchema = new TableSchema(mResultSchema); - setHasResultSet(true); - break; - } - } - } catch (HiveSQLException e) { - setState(OperationState.ERROR); - throw e; - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - } - - private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException { - try { - // In Hive server mode, we are not able to retry in the FetchTask - // case, when calling fetch queries since execute() has returned. - // For now, we disable the test attempts. - driver.setTryCount(Integer.MAX_VALUE); - response = driver.run(); - if (0 != response.getResponseCode()) { - throw toSQLException("Error while processing statement", response); - } - } catch (HiveSQLException e) { - // If the operation was cancelled by another thread, - // Driver#run will return a non-zero response code. - // We will simply return if the operation state is CANCELED, - // otherwise throw an exception - if (getStatus().getState() == OperationState.CANCELED) { - return; - } - else { - setState(OperationState.ERROR); - throw e; - } - } catch (Exception e) { - setState(OperationState.ERROR); - throw new HiveSQLException("Error running query: " + e.toString(), e); - } - setState(OperationState.FINISHED); - } - - @Override - public void runInternal() throws HiveSQLException { - setState(OperationState.PENDING); - final HiveConf opConfig = getConfigForOperation(); - prepare(opConfig); - if (!shouldRunAsync()) { - runQuery(opConfig); - } else { - // We'll pass ThreadLocals in the background thread from the foreground (handler) thread - final SessionState parentSessionState = SessionState.get(); - // ThreadLocal Hive object needs to be set in background thread. - // The metastore client in Hive is associated with right user. - final Hive parentHive = getSessionHive(); - // Current UGI will get used by metastore when metsatore is in embedded mode - // So this needs to get passed to the new background thread - final UserGroupInformation currentUGI = getCurrentUGI(opConfig); - // Runnable impl to call runInternal asynchronously, - // from a different thread - Runnable backgroundOperation = new Runnable() { - @Override - public void run() { - PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { - @Override - public Object run() throws HiveSQLException { - Hive.set(parentHive); - SessionState.setCurrentSessionState(parentSessionState); - // Set current OperationLog in this async thread for keeping on saving query log. - registerCurrentOperationLog(); - try { - runQuery(opConfig); - } catch (HiveSQLException e) { - setOperationException(e); - LOG.error("Error running hive query: ", e); - } finally { - unregisterOperationLog(); - } - return null; - } - }; - - try { - currentUGI.doAs(doAsAction); - } catch (Exception e) { - setOperationException(new HiveSQLException(e)); - LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); - } - finally { - /** - * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup - * when this thread is garbage collected later. - * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() - */ - if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { - ThreadWithGarbageCleanup currentThread = - (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); - currentThread.cacheThreadLocalRawStore(); - } - } - } - }; - try { - // This submit blocks if no background threads are available to run this operation - Future backgroundHandle = - getParentSession().getSessionManager().submitBackgroundOperation(backgroundOperation); - setBackgroundHandle(backgroundHandle); - } catch (RejectedExecutionException rejected) { - setState(OperationState.ERROR); - throw new HiveSQLException("The background threadpool cannot accept" + - " new task for execution, please retry the operation", rejected); - } - } - } - - /** - * Returns the current UGI on the stack - * @param opConfig - * @return UserGroupInformation - * @throws HiveSQLException - */ - private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { - try { - return Utils.getUGI(); - } catch (Exception e) { - throw new HiveSQLException("Unable to get current user", e); - } - } - - /** - * Returns the ThreadLocal Hive for the current thread - * @return Hive - * @throws HiveSQLException - */ - private Hive getSessionHive() throws HiveSQLException { - try { - return Hive.get(); - } catch (HiveException e) { - throw new HiveSQLException("Failed to get ThreadLocal Hive object", e); - } - } - - private void cleanup(OperationState state) throws HiveSQLException { - setState(state); - if (shouldRunAsync()) { - Future backgroundHandle = getBackgroundHandle(); - if (backgroundHandle != null) { - backgroundHandle.cancel(true); - } - } - if (driver != null) { - driver.close(); - driver.destroy(); - } - driver = null; - - SessionState ss = SessionState.get(); - if (ss.getTmpOutputFile() != null) { - ss.getTmpOutputFile().delete(); - } - } - - @Override - public void cancel() throws HiveSQLException { - cleanup(OperationState.CANCELED); - } - - @Override - public void close() throws HiveSQLException { - cleanup(OperationState.CLOSED); - cleanupOperationLog(); - } - - @Override - public TableSchema getResultSetSchema() throws HiveSQLException { - assertState(OperationState.FINISHED); - if (resultSchema == null) { - resultSchema = new TableSchema(driver.getSchema()); - } - return resultSchema; - } - - private final transient List convey = new ArrayList(); - - @Override - public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { - validateDefaultFetchOrientation(orientation); - assertState(OperationState.FINISHED); - - RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); - - try { - /* if client is requesting fetch-from-start and its not the first time reading from this operation - * then reset the fetch position to beginning - */ - if (orientation.equals(FetchOrientation.FETCH_FIRST) && fetchStarted) { - driver.resetFetch(); - } - fetchStarted = true; - driver.setMaxRows((int) maxRows); - if (driver.getResults(convey)) { - return decode(convey, rowSet); - } - return rowSet; - } catch (IOException e) { - throw new HiveSQLException(e); - } catch (CommandNeedRetryException e) { - throw new HiveSQLException(e); - } catch (Exception e) { - throw new HiveSQLException(e); - } finally { - convey.clear(); - } - } - - private RowSet decode(List rows, RowSet rowSet) throws Exception { - if (driver.isFetchingTable()) { - return prepareFromRow(rows, rowSet); - } - return decodeFromString(rows, rowSet); - } - - // already encoded to thrift-able object in ThriftFormatter - private RowSet prepareFromRow(List rows, RowSet rowSet) throws Exception { - for (Object row : rows) { - rowSet.addRow((Object[]) row); - } - return rowSet; - } - - private RowSet decodeFromString(List rows, RowSet rowSet) - throws SQLException, SerDeException { - getSerDe(); - StructObjectInspector soi = (StructObjectInspector) serde.getObjectInspector(); - List fieldRefs = soi.getAllStructFieldRefs(); - - Object[] deserializedFields = new Object[fieldRefs.size()]; - Object rowObj; - ObjectInspector fieldOI; - - int protocol = getProtocolVersion().getValue(); - for (Object rowString : rows) { - rowObj = serde.deserialize(new BytesWritable(((String)rowString).getBytes(StandardCharsets.UTF_8))); - for (int i = 0; i < fieldRefs.size(); i++) { - StructField fieldRef = fieldRefs.get(i); - fieldOI = fieldRef.getFieldObjectInspector(); - Object fieldData = soi.getStructFieldData(rowObj, fieldRef); - deserializedFields[i] = SerDeUtils.toThriftPayload(fieldData, fieldOI, protocol); - } - rowSet.addRow(deserializedFields); - } - return rowSet; - } - - private SerDe getSerDe() throws SQLException { - if (serde != null) { - return serde; - } - try { - List fieldSchemas = mResultSchema.getFieldSchemas(); - StringBuilder namesSb = new StringBuilder(); - StringBuilder typesSb = new StringBuilder(); - - if (fieldSchemas != null && !fieldSchemas.isEmpty()) { - for (int pos = 0; pos < fieldSchemas.size(); pos++) { - if (pos != 0) { - namesSb.append(","); - typesSb.append(","); - } - namesSb.append(fieldSchemas.get(pos).getName()); - typesSb.append(fieldSchemas.get(pos).getType()); - } - } - String names = namesSb.toString(); - String types = typesSb.toString(); - - serde = new LazySimpleSerDe(); - Properties props = new Properties(); - if (names.length() > 0) { - LOG.debug("Column names: " + names); - props.setProperty(serdeConstants.LIST_COLUMNS, names); - } - if (types.length() > 0) { - LOG.debug("Column types: " + types); - props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types); - } - SerDeUtils.initializeSerDe(serde, new HiveConf(), props, null); - - } catch (Exception ex) { - ex.printStackTrace(); - throw new SQLException("Could not create ResultSet: " + ex.getMessage(), ex); - } - return serde; - } - - /** - * If there are query specific settings to overlay, then create a copy of config - * There are two cases we need to clone the session config that's being passed to hive driver - * 1. Async query - - * If the client changes a config setting, that shouldn't reflect in the execution already underway - * 2. confOverlay - - * The query specific settings should only be applied to the query config and not session - * @return new configuration - * @throws HiveSQLException - */ - private HiveConf getConfigForOperation() throws HiveSQLException { - HiveConf sqlOperationConf = getParentSession().getHiveConf(); - if (!getConfOverlay().isEmpty() || shouldRunAsync()) { - // clone the parent session config for this query - sqlOperationConf = new HiveConf(sqlOperationConf); - - // apply overlay query specific settings, if any - for (Map.Entry confEntry : getConfOverlay().entrySet()) { - try { - sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); - } catch (IllegalArgumentException e) { - throw new HiveSQLException("Error applying statement specific settings", e); - } - } - } - return sqlOperationConf; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java deleted file mode 100644 index e59d19ea6be42..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * 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.hive.service.cli.operation; - -import java.util.Set; - - -public interface TableTypeMapping { - /** - * Map client's table type name to hive's table type - * @param clientTypeName - * @return - */ - String mapToHiveType(String clientTypeName); - - /** - * Map hive's table type name to client's table type - * @param hiveTypeName - * @return - */ - String mapToClientType(String hiveTypeName); - - /** - * Get all the table types of this mapping - * @return - */ - Set getTableTypeNames(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java deleted file mode 100644 index 65f9b298bf4f6..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java +++ /dev/null @@ -1,156 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.*; - -public interface HiveSession extends HiveSessionBase { - - void open(Map sessionConfMap) throws Exception; - - IMetaStoreClient getMetaStoreClient() throws HiveSQLException; - - /** - * getInfo operation handler - * @param getInfoType - * @return - * @throws HiveSQLException - */ - GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException; - - /** - * execute operation handler - * @param statement - * @param confOverlay - * @return - * @throws HiveSQLException - */ - OperationHandle executeStatement(String statement, - Map confOverlay) throws HiveSQLException; - - /** - * execute operation handler - * @param statement - * @param confOverlay - * @return - * @throws HiveSQLException - */ - OperationHandle executeStatementAsync(String statement, - Map confOverlay) throws HiveSQLException; - - /** - * getTypeInfo operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getTypeInfo() throws HiveSQLException; - - /** - * getCatalogs operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getCatalogs() throws HiveSQLException; - - /** - * getSchemas operation handler - * @param catalogName - * @param schemaName - * @return - * @throws HiveSQLException - */ - OperationHandle getSchemas(String catalogName, String schemaName) - throws HiveSQLException; - - /** - * getTables operation handler - * @param catalogName - * @param schemaName - * @param tableName - * @param tableTypes - * @return - * @throws HiveSQLException - */ - OperationHandle getTables(String catalogName, String schemaName, - String tableName, List tableTypes) throws HiveSQLException; - - /** - * getTableTypes operation handler - * @return - * @throws HiveSQLException - */ - OperationHandle getTableTypes() throws HiveSQLException ; - - /** - * getColumns operation handler - * @param catalogName - * @param schemaName - * @param tableName - * @param columnName - * @return - * @throws HiveSQLException - */ - OperationHandle getColumns(String catalogName, String schemaName, - String tableName, String columnName) throws HiveSQLException; - - /** - * getFunctions operation handler - * @param catalogName - * @param schemaName - * @param functionName - * @return - * @throws HiveSQLException - */ - OperationHandle getFunctions(String catalogName, String schemaName, - String functionName) throws HiveSQLException; - - /** - * close the session - * @throws HiveSQLException - */ - void close() throws HiveSQLException; - - void cancelOperation(OperationHandle opHandle) throws HiveSQLException; - - void closeOperation(OperationHandle opHandle) throws HiveSQLException; - - TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException; - - RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException; - - String getDelegationToken(HiveAuthFactory authFactory, String owner, - String renewer) throws HiveSQLException; - - void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException; - - void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException; - - void closeExpiredOperations(); - - long getNoOperationTime(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java deleted file mode 100644 index b72c18b2b2135..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -import java.io.File; - -/** - * Methods that don't need to be executed under a doAs - * context are here. Rest of them in HiveSession interface - */ -public interface HiveSessionBase { - - TProtocolVersion getProtocolVersion(); - - /** - * Set the session manager for the session - * @param sessionManager - */ - void setSessionManager(SessionManager sessionManager); - - /** - * Get the session manager for the session - */ - SessionManager getSessionManager(); - - /** - * Set operation manager for the session - * @param operationManager - */ - void setOperationManager(OperationManager operationManager); - - /** - * Check whether operation logging is enabled and session dir is created successfully - */ - boolean isOperationLogEnabled(); - - /** - * Get the session dir, which is the parent dir of operation logs - * @return a file representing the parent directory of operation logs - */ - File getOperationLogSessionDir(); - - /** - * Set the session dir, which is the parent dir of operation logs - * @param operationLogRootDir the parent dir of the session dir - */ - void setOperationLogSessionDir(File operationLogRootDir); - - SessionHandle getSessionHandle(); - - String getUsername(); - - String getPassword(); - - HiveConf getHiveConf(); - - SessionState getSessionState(); - - String getUserName(); - - void setUserName(String userName); - - String getIpAddress(); - - void setIpAddress(String ipAddress); - - long getLastAccessTime(); -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java deleted file mode 100644 index e3fb54d9f47e9..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ /dev/null @@ -1,842 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.common.cli.HiveFileProcessor; -import org.apache.hadoop.hive.common.cli.IHiveFileProcessor; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.exec.FetchFormatter; -import org.apache.hadoop.hive.ql.exec.ListSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.history.HiveHistory; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.common.util.HiveVersionInfo; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.FetchType; -import org.apache.hive.service.cli.GetInfoType; -import org.apache.hive.service.cli.GetInfoValue; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.TableSchema; -import org.apache.hive.service.cli.operation.ExecuteStatementOperation; -import org.apache.hive.service.cli.operation.GetCatalogsOperation; -import org.apache.hive.service.cli.operation.GetColumnsOperation; -import org.apache.hive.service.cli.operation.GetFunctionsOperation; -import org.apache.hive.service.cli.operation.GetSchemasOperation; -import org.apache.hive.service.cli.operation.GetTableTypesOperation; -import org.apache.hive.service.cli.operation.GetTypeInfoOperation; -import org.apache.hive.service.cli.operation.MetadataOperation; -import org.apache.hive.service.cli.operation.Operation; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.ThreadWithGarbageCleanup; - -import static org.apache.hadoop.hive.conf.SystemVariables.ENV_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.HIVECONF_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.HIVEVAR_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.METACONF_PREFIX; -import static org.apache.hadoop.hive.conf.SystemVariables.SYSTEM_PREFIX; - -/** - * HiveSession - * - */ -public class HiveSessionImpl implements HiveSession { - private final SessionHandle sessionHandle; - private String username; - private final String password; - private HiveConf hiveConf; - private SessionState sessionState; - private String ipAddress; - private static final String FETCH_WORK_SERDE_CLASS = - "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; - private static final Log LOG = LogFactory.getLog(HiveSessionImpl.class); - private SessionManager sessionManager; - private OperationManager operationManager; - private final Set opHandleSet = new HashSet(); - private boolean isOperationLogEnabled; - private File sessionLogDir; - private volatile long lastAccessTime; - private volatile long lastIdleTime; - - public HiveSessionImpl(TProtocolVersion protocol, String username, String password, - HiveConf serverhiveConf, String ipAddress) { - this.username = username; - this.password = password; - this.sessionHandle = new SessionHandle(protocol); - this.hiveConf = new HiveConf(serverhiveConf); - this.ipAddress = ipAddress; - - try { - // In non-impersonation mode, map scheduler queue to current user - // if fair scheduler is configured. - if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { - ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); - } - } catch (IOException e) { - LOG.warn("Error setting scheduler queue: " + e, e); - } - // Set an explicit session name to control the download directory name - hiveConf.set(ConfVars.HIVESESSIONID.varname, - sessionHandle.getHandleIdentifier().toString()); - // Use thrift transportable formatter - hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER, - FetchFormatter.ThriftFormatter.class.getName()); - hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue()); - } - - @Override - /** - * Opens a new HiveServer2 session for the client connection. - * Creates a new SessionState object that will be associated with this HiveServer2 session. - * When the server executes multiple queries in the same session, - * this SessionState object is reused across multiple queries. - * Note that if doAs is true, this call goes through a proxy object, - * which wraps the method logic in a UserGroupInformation#doAs. - * That's why it is important to create SessionState here rather than in the constructor. - */ - public void open(Map sessionConfMap) throws HiveSQLException { - sessionState = new SessionState(hiveConf, username); - sessionState.setUserIpAddress(ipAddress); - sessionState.setIsHiveServerQuery(true); - SessionState.start(sessionState); - try { - sessionState.reloadAuxJars(); - } catch (IOException e) { - String msg = "Failed to load reloadable jar file path: " + e; - LOG.error(msg, e); - throw new HiveSQLException(msg, e); - } - // Process global init file: .hiverc - processGlobalInitFile(); - if (sessionConfMap != null) { - configureSession(sessionConfMap); - } - lastAccessTime = System.currentTimeMillis(); - lastIdleTime = lastAccessTime; - } - - /** - * It is used for processing hiverc file from HiveServer2 side. - */ - private class GlobalHivercFileProcessor extends HiveFileProcessor { - @Override - protected BufferedReader loadFile(String fileName) throws IOException { - FileInputStream initStream = null; - BufferedReader bufferedReader = null; - initStream = new FileInputStream(fileName); - bufferedReader = new BufferedReader(new InputStreamReader(initStream)); - return bufferedReader; - } - - @Override - protected int processCmd(String cmd) { - int rc = 0; - String cmd_trimed = cmd.trim(); - try { - executeStatementInternal(cmd_trimed, null, false); - } catch (HiveSQLException e) { - rc = -1; - LOG.warn("Failed to execute HQL command in global .hiverc file.", e); - } - return rc; - } - } - - private void processGlobalInitFile() { - IHiveFileProcessor processor = new GlobalHivercFileProcessor(); - - try { - String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); - if (hiverc != null) { - File hivercFile = new File(hiverc); - if (hivercFile.isDirectory()) { - hivercFile = new File(hivercFile, SessionManager.HIVERCFILE); - } - if (hivercFile.isFile()) { - LOG.info("Running global init file: " + hivercFile); - int rc = processor.processFile(hivercFile.getAbsolutePath()); - if (rc != 0) { - LOG.error("Failed on initializing global .hiverc file"); - } - } else { - LOG.debug("Global init file " + hivercFile + " does not exist"); - } - } - } catch (IOException e) { - LOG.warn("Failed on initializing global .hiverc file", e); - } - } - - private void configureSession(Map sessionConfMap) throws HiveSQLException { - SessionState.setCurrentSessionState(sessionState); - for (Map.Entry entry : sessionConfMap.entrySet()) { - String key = entry.getKey(); - if (key.startsWith("set:")) { - try { - setVariable(key.substring(4), entry.getValue()); - } catch (Exception e) { - throw new HiveSQLException(e); - } - } else if (key.startsWith("use:")) { - SessionState.get().setCurrentDatabase(entry.getValue()); - } else { - hiveConf.verifyAndSet(key, entry.getValue()); - } - } - } - - // Copy from org.apache.hadoop.hive.ql.processors.SetProcessor, only change: - // setConf(varname, propName, varvalue, true) when varname.startsWith(HIVECONF_PREFIX) - public static int setVariable(String varname, String varvalue) throws Exception { - SessionState ss = SessionState.get(); - if (varvalue.contains("\n")){ - ss.err.println("Warning: Value had a \\n character in it."); - } - varname = varname.trim(); - if (varname.startsWith(ENV_PREFIX)){ - ss.err.println("env:* variables can not be set."); - return 1; - } else if (varname.startsWith(SYSTEM_PREFIX)){ - String propName = varname.substring(SYSTEM_PREFIX.length()); - System.getProperties().setProperty(propName, - new VariableSubstitution().substitute(ss.getConf(),varvalue)); - } else if (varname.startsWith(HIVECONF_PREFIX)){ - String propName = varname.substring(HIVECONF_PREFIX.length()); - setConf(varname, propName, varvalue, true); - } else if (varname.startsWith(HIVEVAR_PREFIX)) { - String propName = varname.substring(HIVEVAR_PREFIX.length()); - ss.getHiveVariables().put(propName, - new VariableSubstitution().substitute(ss.getConf(),varvalue)); - } else if (varname.startsWith(METACONF_PREFIX)) { - String propName = varname.substring(METACONF_PREFIX.length()); - Hive hive = Hive.get(ss.getConf()); - hive.setMetaConf(propName, new VariableSubstitution().substitute(ss.getConf(), varvalue)); - } else { - setConf(varname, varname, varvalue, true); - } - return 0; - } - - // returns non-null string for validation fail - private static void setConf(String varname, String key, String varvalue, boolean register) - throws IllegalArgumentException { - HiveConf conf = SessionState.get().getConf(); - String value = new VariableSubstitution().substitute(conf, varvalue); - if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { - HiveConf.ConfVars confVars = HiveConf.getConfVars(key); - if (confVars != null) { - if (!confVars.isType(value)) { - StringBuilder message = new StringBuilder(); - message.append("'SET ").append(varname).append('=').append(varvalue); - message.append("' FAILED because ").append(key).append(" expects "); - message.append(confVars.typeString()).append(" type value."); - throw new IllegalArgumentException(message.toString()); - } - String fail = confVars.validate(value); - if (fail != null) { - StringBuilder message = new StringBuilder(); - message.append("'SET ").append(varname).append('=').append(varvalue); - message.append("' FAILED in validation : ").append(fail).append('.'); - throw new IllegalArgumentException(message.toString()); - } - } else if (key.startsWith("hive.")) { - throw new IllegalArgumentException("hive configuration " + key + " does not exists."); - } - } - conf.verifyAndSet(key, value); - if (register) { - SessionState.get().getOverriddenConfigurations().put(key, value); - } - } - - @Override - public void setOperationLogSessionDir(File operationLogRootDir) { - if (!operationLogRootDir.exists()) { - LOG.warn("The operation log root directory is removed, recreating: " + - operationLogRootDir.getAbsolutePath()); - if (!operationLogRootDir.mkdirs()) { - LOG.warn("Unable to create operation log root directory: " + - operationLogRootDir.getAbsolutePath()); - } - } - if (!operationLogRootDir.canWrite()) { - LOG.warn("The operation log root directory is not writable: " + - operationLogRootDir.getAbsolutePath()); - } - sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); - isOperationLogEnabled = true; - if (!sessionLogDir.exists()) { - if (!sessionLogDir.mkdir()) { - LOG.warn("Unable to create operation log session directory: " + - sessionLogDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - } - if (isOperationLogEnabled) { - LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath()); - } - } - - @Override - public boolean isOperationLogEnabled() { - return isOperationLogEnabled; - } - - @Override - public File getOperationLogSessionDir() { - return sessionLogDir; - } - - @Override - public TProtocolVersion getProtocolVersion() { - return sessionHandle.getProtocolVersion(); - } - - @Override - public SessionManager getSessionManager() { - return sessionManager; - } - - @Override - public void setSessionManager(SessionManager sessionManager) { - this.sessionManager = sessionManager; - } - - private OperationManager getOperationManager() { - return operationManager; - } - - @Override - public void setOperationManager(OperationManager operationManager) { - this.operationManager = operationManager; - } - - protected synchronized void acquire(boolean userAccess) { - // Need to make sure that the this HiveServer2's session's SessionState is - // stored in the thread local for the handler thread. - SessionState.setCurrentSessionState(sessionState); - if (userAccess) { - lastAccessTime = System.currentTimeMillis(); - } - } - - /** - * 1. We'll remove the ThreadLocal SessionState as this thread might now serve - * other requests. - * 2. We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup - * when this thread is garbage collected later. - * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() - */ - protected synchronized void release(boolean userAccess) { - SessionState.detachSession(); - if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { - ThreadWithGarbageCleanup currentThread = - (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); - currentThread.cacheThreadLocalRawStore(); - } - if (userAccess) { - lastAccessTime = System.currentTimeMillis(); - } - if (opHandleSet.isEmpty()) { - lastIdleTime = System.currentTimeMillis(); - } else { - lastIdleTime = 0; - } - } - - @Override - public SessionHandle getSessionHandle() { - return sessionHandle; - } - - @Override - public String getUsername() { - return username; - } - - @Override - public String getPassword() { - return password; - } - - @Override - public HiveConf getHiveConf() { - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); - return hiveConf; - } - - @Override - public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { - try { - return Hive.get(getHiveConf()).getMSC(); - } catch (HiveException e) { - throw new HiveSQLException("Failed to get metastore connection", e); - } catch (MetaException e) { - throw new HiveSQLException("Failed to get metastore connection", e); - } - } - - @Override - public GetInfoValue getInfo(GetInfoType getInfoType) - throws HiveSQLException { - acquire(true); - try { - switch (getInfoType) { - case CLI_SERVER_NAME: - return new GetInfoValue("Hive"); - case CLI_DBMS_NAME: - return new GetInfoValue("Apache Hive"); - case CLI_DBMS_VER: - return new GetInfoValue(HiveVersionInfo.getVersion()); - case CLI_MAX_COLUMN_NAME_LEN: - return new GetInfoValue(128); - case CLI_MAX_SCHEMA_NAME_LEN: - return new GetInfoValue(128); - case CLI_MAX_TABLE_NAME_LEN: - return new GetInfoValue(128); - case CLI_TXN_CAPABLE: - default: - throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); - } - } finally { - release(true); - } - } - - @Override - public OperationHandle executeStatement(String statement, Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(statement, confOverlay, false); - } - - @Override - public OperationHandle executeStatementAsync(String statement, Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(statement, confOverlay, true); - } - - private OperationHandle executeStatementInternal(String statement, Map confOverlay, - boolean runAsync) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - ExecuteStatementOperation operation = operationManager - .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - // Referring to SQLOperation.java, there is no chance that a HiveSQLException throws and the asyn - // background operation submits to thread pool successfully at the same time. So, Cleanup - // opHandle directly when got HiveSQLException - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTypeInfo() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getCatalogs() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getSchemas(String catalogName, String schemaName) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetSchemasOperation operation = - operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTables(String catalogName, String schemaName, String tableName, - List tableTypes) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - MetadataOperation operation = - operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getTableTypes() - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getColumns(String catalogName, String schemaName, - String tableName, String columnName) throws HiveSQLException { - acquire(true); - String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); - if (StringUtils.isNotBlank(addedJars)) { - IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); - metastoreClient.setHiveAddedJars(addedJars); - } - OperationManager operationManager = getOperationManager(); - GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(), - catalogName, schemaName, tableName, columnName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) - throws HiveSQLException { - acquire(true); - - OperationManager operationManager = getOperationManager(); - GetFunctionsOperation operation = operationManager - .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName); - OperationHandle opHandle = operation.getHandle(); - try { - operation.run(); - opHandleSet.add(opHandle); - return opHandle; - } catch (HiveSQLException e) { - operationManager.closeOperation(opHandle); - throw e; - } finally { - release(true); - } - } - - @Override - public void close() throws HiveSQLException { - try { - acquire(true); - // Iterate through the opHandles and close their operations - for (OperationHandle opHandle : opHandleSet) { - try { - operationManager.closeOperation(opHandle); - } catch (Exception e) { - LOG.warn("Exception is thrown closing operation " + opHandle, e); - } - } - opHandleSet.clear(); - // Cleanup session log directory. - cleanupSessionLogDir(); - // Cleanup pipeout file. - cleanupPipeoutFile(); - HiveHistory hiveHist = sessionState.getHiveHistory(); - if (null != hiveHist) { - hiveHist.closeStream(); - } - try { - sessionState.close(); - } finally { - sessionState = null; - } - } catch (IOException ioe) { - throw new HiveSQLException("Failure to close", ioe); - } finally { - if (sessionState != null) { - try { - sessionState.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - sessionState = null; - } - release(true); - } - } - - private void cleanupPipeoutFile() { - String lScratchDir = hiveConf.getVar(ConfVars.LOCALSCRATCHDIR); - String sessionID = hiveConf.getVar(ConfVars.HIVESESSIONID); - - File[] fileAry = new File(lScratchDir).listFiles( - (dir, name) -> name.startsWith(sessionID) && name.endsWith(".pipeout")); - - if (fileAry == null) { - LOG.error("Unable to access pipeout files in " + lScratchDir); - } else { - for (File file : fileAry) { - try { - FileUtils.forceDelete(file); - } catch (Exception e) { - LOG.error("Failed to cleanup pipeout file: " + file, e); - } - } - } - } - - private void cleanupSessionLogDir() { - if (isOperationLogEnabled) { - try { - FileUtils.forceDelete(sessionLogDir); - } catch (Exception e) { - LOG.error("Failed to cleanup session log dir: " + sessionHandle, e); - } - } - } - - @Override - public SessionState getSessionState() { - return sessionState; - } - - @Override - public String getUserName() { - return username; - } - - @Override - public void setUserName(String userName) { - this.username = userName; - } - - @Override - public long getLastAccessTime() { - return lastAccessTime; - } - - @Override - public void closeExpiredOperations() { - OperationHandle[] handles = opHandleSet.toArray(new OperationHandle[opHandleSet.size()]); - if (handles.length > 0) { - List operations = operationManager.removeExpiredOperations(handles); - if (!operations.isEmpty()) { - closeTimedOutOperations(operations); - } - } - } - - @Override - public long getNoOperationTime() { - return lastIdleTime > 0 ? System.currentTimeMillis() - lastIdleTime : 0; - } - - private void closeTimedOutOperations(List operations) { - acquire(false); - try { - for (Operation operation : operations) { - opHandleSet.remove(operation.getHandle()); - try { - operation.close(); - } catch (Exception e) { - LOG.warn("Exception is thrown closing timed-out operation " + operation.getHandle(), e); - } - } - } finally { - release(false); - } - } - - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - sessionManager.getOperationManager().cancelOperation(opHandle); - } finally { - release(true); - } - } - - @Override - public void closeOperation(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - operationManager.closeOperation(opHandle); - opHandleSet.remove(opHandle); - } finally { - release(true); - } - } - - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { - acquire(true); - try { - return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); - } finally { - release(true); - } - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, - long maxRows, FetchType fetchType) throws HiveSQLException { - acquire(true); - try { - if (fetchType == FetchType.QUERY_OUTPUT) { - return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); - } - return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows); - } finally { - release(true); - } - } - - protected HiveSession getSession() { - return this; - } - - @Override - public String getIpAddress() { - return ipAddress; - } - - @Override - public void setIpAddress(String ipAddress) { - this.ipAddress = ipAddress; - } - - @Override - public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf()); - return authFactory.getDelegationToken(owner, renewer); - } - - @Override - public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), - getIpAddress(), getHiveConf()); - authFactory.cancelDelegationToken(tokenStr); - } - - @Override - public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), - getIpAddress(), getHiveConf()); - authFactory.renewDelegationToken(tokenStr); - } - - // extract the real user from the given token string - private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { - return authFactory.getUserFromToken(tokenStr); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java deleted file mode 100644 index 762dbb2faadec..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.thrift.TProtocolVersion; - -/** - * - * HiveSessionImplwithUGI. - * HiveSession with connecting user's UGI and delegation token if required - */ -public class HiveSessionImplwithUGI extends HiveSessionImpl { - public static final String HS2TOKEN = "HiveServer2ImpersonationToken"; - - private UserGroupInformation sessionUgi = null; - private String delegationTokenStr = null; - private Hive sessionHive = null; - private HiveSession proxySession = null; - static final Log LOG = LogFactory.getLog(HiveSessionImplwithUGI.class); - - public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password, - HiveConf hiveConf, String ipAddress, String delegationToken) throws HiveSQLException { - super(protocol, username, password, hiveConf, ipAddress); - setSessionUGI(username); - setDelegationToken(delegationToken); - - // create a new metastore connection for this particular user session - Hive.set(null); - try { - sessionHive = Hive.get(getHiveConf()); - } catch (HiveException e) { - throw new HiveSQLException("Failed to setup metastore connection", e); - } - } - - // setup appropriate UGI for the session - public void setSessionUGI(String owner) throws HiveSQLException { - if (owner == null) { - throw new HiveSQLException("No username provided for impersonation"); - } - if (UserGroupInformation.isSecurityEnabled()) { - try { - sessionUgi = UserGroupInformation.createProxyUser( - owner, UserGroupInformation.getLoginUser()); - } catch (IOException e) { - throw new HiveSQLException("Couldn't setup proxy user", e); - } - } else { - sessionUgi = UserGroupInformation.createRemoteUser(owner); - } - } - - public UserGroupInformation getSessionUgi() { - return this.sessionUgi; - } - - public String getDelegationToken() { - return this.delegationTokenStr; - } - - @Override - protected synchronized void acquire(boolean userAccess) { - super.acquire(userAccess); - // if we have a metastore connection with impersonation, then set it first - if (sessionHive != null) { - Hive.set(sessionHive); - } - } - - /** - * Close the file systems for the session and remove it from the FileSystem cache. - * Cancel the session's delegation token and close the metastore connection - */ - @Override - public void close() throws HiveSQLException { - try { - acquire(true); - cancelDelegationToken(); - } finally { - try { - super.close(); - } finally { - try { - FileSystem.closeAllForUGI(sessionUgi); - } catch (IOException ioe) { - throw new HiveSQLException("Could not clean up file-system handles for UGI: " - + sessionUgi, ioe); - } - } - } - } - - /** - * Enable delegation token for the session - * save the token string and set the token.signature in hive conf. The metastore client uses - * this token.signature to determine where to use kerberos or delegation token - * @throws HiveException - * @throws IOException - */ - private void setDelegationToken(String delegationTokenStr) throws HiveSQLException { - this.delegationTokenStr = delegationTokenStr; - if (delegationTokenStr != null) { - getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); - try { - Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); - } catch (IOException e) { - throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); - } - } - } - - // If the session has a delegation token obtained from the metastore, then cancel it - private void cancelDelegationToken() throws HiveSQLException { - if (delegationTokenStr != null) { - try { - Hive.get(getHiveConf()).cancelDelegationToken(delegationTokenStr); - } catch (HiveException e) { - throw new HiveSQLException("Couldn't cancel delegation token", e); - } - // close the metastore connection created with this delegation token - Hive.closeCurrent(); - } - } - - @Override - protected HiveSession getSession() { - assert proxySession != null; - - return proxySession; - } - - public void setProxySession(HiveSession proxySession) { - this.proxySession = proxySession; - } - - @Override - public String getDelegationToken(HiveAuthFactory authFactory, String owner, - String renewer) throws HiveSQLException { - return authFactory.getDelegationToken(owner, renewer); - } - - @Override - public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - authFactory.cancelDelegationToken(tokenStr); - } - - @Override - public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) - throws HiveSQLException { - authFactory.renewDelegationToken(tokenStr); - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java deleted file mode 100644 index ad6fb3ba37a0e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ /dev/null @@ -1,377 +0,0 @@ -/** - * 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.hive.service.cli.session; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Date; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.operation.OperationManager; -import org.apache.hive.service.cli.thrift.TProtocolVersion; -import org.apache.hive.service.server.HiveServer2; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; - -/** - * SessionManager. - * - */ -public class SessionManager extends CompositeService { - - private static final Log LOG = LogFactory.getLog(SessionManager.class); - public static final String HIVERCFILE = ".hiverc"; - private HiveConf hiveConf; - private final Map handleToSession = - new ConcurrentHashMap(); - private final OperationManager operationManager = new OperationManager(); - private ThreadPoolExecutor backgroundOperationPool; - private boolean isOperationLogEnabled; - private File operationLogRootDir; - - private long checkInterval; - private long sessionTimeout; - private boolean checkOperation; - - private volatile boolean shutdown; - // The HiveServer2 instance running this service - private final HiveServer2 hiveServer2; - - public SessionManager(HiveServer2 hiveServer2) { - super(SessionManager.class.getSimpleName()); - this.hiveServer2 = hiveServer2; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - //Create operation log root directory, if operation logging is enabled - if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { - initOperationLogRootDir(); - } - createBackgroundOperationPool(); - addService(operationManager); - super.init(hiveConf); - } - - private void createBackgroundOperationPool() { - int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); - LOG.info("HiveServer2: Background operation thread pool size: " + poolSize); - int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); - LOG.info("HiveServer2: Background operation thread wait queue size: " + poolQueueSize); - long keepAliveTime = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); - LOG.info( - "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds"); - - // Create a thread pool with #poolSize threads - // Threads terminate when they are idle for more than the keepAliveTime - // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize - String threadPoolName = "HiveServer2-Background-Pool"; - backgroundOperationPool = new ThreadPoolExecutor(poolSize, poolSize, - keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(poolQueueSize), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - backgroundOperationPool.allowCoreThreadTimeOut(true); - - checkInterval = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL, TimeUnit.MILLISECONDS); - sessionTimeout = HiveConf.getTimeVar( - hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS); - checkOperation = HiveConf.getBoolVar(hiveConf, - ConfVars.HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION); - } - - private void initOperationLogRootDir() { - operationLogRootDir = new File( - hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); - isOperationLogEnabled = true; - - if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { - LOG.warn("The operation log root directory exists, but it is not a directory: " + - operationLogRootDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - - if (!operationLogRootDir.exists()) { - if (!operationLogRootDir.mkdirs()) { - LOG.warn("Unable to create operation log root directory: " + - operationLogRootDir.getAbsolutePath()); - isOperationLogEnabled = false; - } - } - - if (isOperationLogEnabled) { - LOG.info("Operation log root directory is created: " + operationLogRootDir.getAbsolutePath()); - try { - FileUtils.forceDeleteOnExit(operationLogRootDir); - } catch (IOException e) { - LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: " + - operationLogRootDir.getAbsolutePath(), e); - } - } - } - - @Override - public synchronized void start() { - super.start(); - if (checkInterval > 0) { - startTimeoutChecker(); - } - } - - private final Object timeoutCheckerLock = new Object(); - - private void startTimeoutChecker() { - final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds - final Runnable timeoutChecker = new Runnable() { - @Override - public void run() { - sleepFor(interval); - while (!shutdown) { - long current = System.currentTimeMillis(); - for (HiveSession session : new ArrayList(handleToSession.values())) { - if (shutdown) { - break; - } - if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current - && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { - SessionHandle handle = session.getSessionHandle(); - LOG.warn("Session " + handle + " is Timed-out (last access : " + - new Date(session.getLastAccessTime()) + ") and will be closed"); - try { - closeSession(handle); - } catch (HiveSQLException e) { - LOG.warn("Exception is thrown closing session " + handle, e); - } - } else { - session.closeExpiredOperations(); - } - } - sleepFor(interval); - } - } - - private void sleepFor(long interval) { - synchronized (timeoutCheckerLock) { - try { - timeoutCheckerLock.wait(interval); - } catch (InterruptedException e) { - // Ignore, and break. - } - } - } - }; - backgroundOperationPool.execute(timeoutChecker); - } - - private void shutdownTimeoutChecker() { - shutdown = true; - synchronized (timeoutCheckerLock) { - timeoutCheckerLock.notify(); - } - } - - @Override - public synchronized void stop() { - super.stop(); - shutdownTimeoutChecker(); - if (backgroundOperationPool != null) { - backgroundOperationPool.shutdown(); - long timeout = hiveConf.getTimeVar( - ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); - try { - backgroundOperationPool.awaitTermination(timeout, TimeUnit.SECONDS); - } catch (InterruptedException e) { - LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = " + timeout + - " seconds has been exceeded. RUNNING background operations will be shut down", e); - } - backgroundOperationPool = null; - } - cleanupLoggingRootDir(); - } - - private void cleanupLoggingRootDir() { - if (isOperationLogEnabled) { - try { - FileUtils.forceDelete(operationLogRootDir); - } catch (Exception e) { - LOG.warn("Failed to cleanup root dir of HS2 logging: " + operationLogRootDir - .getAbsolutePath(), e); - } - } - } - - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map sessionConf) throws HiveSQLException { - return openSession(protocol, username, password, ipAddress, sessionConf, false, null); - } - - /** - * Opens a new session and creates a session handle. - * The username passed to this method is the effective username. - * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession - * within a UGI.doAs, where UGI corresponds to the effective user. - * - * Please see {@code org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName()} for - * more details. - * - * @param protocol - * @param username - * @param password - * @param ipAddress - * @param sessionConf - * @param withImpersonation - * @param delegationToken - * @return - * @throws HiveSQLException - */ - public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, - Map sessionConf, boolean withImpersonation, String delegationToken) - throws HiveSQLException { - HiveSession session; - // If doAs is set to true for HiveServer2, we will create a proxy object for the session impl. - // Within the proxy object, we wrap the method call in a UserGroupInformation#doAs - if (withImpersonation) { - HiveSessionImplwithUGI sessionWithUGI = new HiveSessionImplwithUGI(protocol, username, password, - hiveConf, ipAddress, delegationToken); - session = HiveSessionProxy.getProxy(sessionWithUGI, sessionWithUGI.getSessionUgi()); - sessionWithUGI.setProxySession(session); - } else { - session = new HiveSessionImpl(protocol, username, password, hiveConf, ipAddress); - } - session.setSessionManager(this); - session.setOperationManager(operationManager); - try { - session.open(sessionConf); - } catch (Exception e) { - try { - session.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - session = null; - throw new HiveSQLException("Failed to open new session: " + e, e); - } - if (isOperationLogEnabled) { - session.setOperationLogSessionDir(operationLogRootDir); - } - handleToSession.put(session.getSessionHandle(), session); - return session.getSessionHandle(); - } - - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - HiveSession session = handleToSession.remove(sessionHandle); - if (session == null) { - throw new HiveSQLException("Session does not exist!"); - } - session.close(); - } - - public HiveSession getSession(SessionHandle sessionHandle) throws HiveSQLException { - HiveSession session = handleToSession.get(sessionHandle); - if (session == null) { - throw new HiveSQLException("Invalid SessionHandle: " + sessionHandle); - } - return session; - } - - public OperationManager getOperationManager() { - return operationManager; - } - - private static ThreadLocal threadLocalIpAddress = new ThreadLocal() { - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setIpAddress(String ipAddress) { - threadLocalIpAddress.set(ipAddress); - } - - public static void clearIpAddress() { - threadLocalIpAddress.remove(); - } - - public static String getIpAddress() { - return threadLocalIpAddress.get(); - } - - private static ThreadLocal threadLocalUserName = new ThreadLocal(){ - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setUserName(String userName) { - threadLocalUserName.set(userName); - } - - public static void clearUserName() { - threadLocalUserName.remove(); - } - - public static String getUserName() { - return threadLocalUserName.get(); - } - - private static ThreadLocal threadLocalProxyUserName = new ThreadLocal(){ - @Override - protected synchronized String initialValue() { - return null; - } - }; - - public static void setProxyUserName(String userName) { - LOG.debug("setting proxy user name based on query param to: " + userName); - threadLocalProxyUserName.set(userName); - } - - public static String getProxyUserName() { - return threadLocalProxyUserName.get(); - } - - public static void clearProxyUserName() { - threadLocalProxyUserName.remove(); - } - - public Future submitBackgroundOperation(Runnable r) { - return backgroundOperationPool.submit(r); - } - - public int getOpenSessionCount() { - return handleToSession.size(); - } -} - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java deleted file mode 100644 index 00bdf7e19126e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ /dev/null @@ -1,121 +0,0 @@ -/** - * 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.hive.service.cli.thrift; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; -import org.apache.thrift.TProcessorFactory; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.server.TThreadPoolServer; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TTransportFactory; - - -public class ThriftBinaryCLIService extends ThriftCLIService { - - public ThriftBinaryCLIService(CLIService cliService) { - super(cliService, ThriftBinaryCLIService.class.getSimpleName()); - } - - @Override - protected void initializeServer() { - try { - // Server thread pool - String threadPoolName = "HiveServer2-Handler-Pool"; - ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, - workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - - // Thrift configs - hiveAuthFactory = new HiveAuthFactory(hiveConf); - TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory(); - TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this); - TServerSocket serverSocket = null; - List sslVersionBlacklist = new ArrayList(); - for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) { - sslVersionBlacklist.add(sslVersion); - } - if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) { - serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum); - } else { - String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); - if (keyStorePath.isEmpty()) { - throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname - + " Not configured for SSL connection"); - } - String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); - serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum, keyStorePath, - keyStorePassword, sslVersionBlacklist); - } - - // In case HIVE_SERVER2_THRIFT_PORT or hive.server2.thrift.port is configured with 0 which - // represents any free port, we should set it to the actual one - portNum = serverSocket.getServerSocket().getLocalPort(); - - // Server args - int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); - int requestTimeout = (int) hiveConf.getTimeVar( - HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); - int beBackoffSlotLength = (int) hiveConf.getTimeVar( - HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); - TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) - .processorFactory(processorFactory).transportFactory(transportFactory) - .protocolFactory(new TBinaryProtocol.Factory()) - .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) - .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) - .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) - .executorService(executorService); - - // TCP Server - server = new TThreadPoolServer(sargs); - server.setServerEventHandler(serverEventHandler); - String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " - + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; - LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { - server.serve(); - } catch (Throwable t) { - LOG.fatal( - "Error starting HiveServer2: could not start " - + ThriftBinaryCLIService.class.getSimpleName(), t); - System.exit(-1); - } - } - -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java deleted file mode 100644 index ff533769b5b84..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ /dev/null @@ -1,693 +0,0 @@ -/** - * 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.hive.service.cli.thrift; - -import javax.security.auth.login.LoginException; -import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.service.AbstractService; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.ServiceUtils; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.*; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.server.HiveServer2; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TProtocol; -import org.apache.thrift.server.ServerContext; -import org.apache.thrift.server.TServer; -import org.apache.thrift.server.TServerEventHandler; -import org.apache.thrift.transport.TTransport; - -/** - * ThriftCLIService. - * - */ -public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable { - - public static final Log LOG = LogFactory.getLog(ThriftCLIService.class.getName()); - - protected CLIService cliService; - private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); - protected static HiveAuthFactory hiveAuthFactory; - - protected int portNum; - protected InetAddress serverIPAddress; - protected String hiveHost; - protected TServer server; - protected org.eclipse.jetty.server.Server httpServer; - - private boolean isStarted = false; - protected boolean isEmbedded = false; - - protected HiveConf hiveConf; - - protected int minWorkerThreads; - protected int maxWorkerThreads; - protected long workerKeepAliveTime; - - protected TServerEventHandler serverEventHandler; - protected ThreadLocal currentServerContext; - - static class ThriftCLIServerContext implements ServerContext { - private SessionHandle sessionHandle = null; - - public void setSessionHandle(SessionHandle sessionHandle) { - this.sessionHandle = sessionHandle; - } - - public SessionHandle getSessionHandle() { - return sessionHandle; - } - } - - public ThriftCLIService(CLIService service, String serviceName) { - super(serviceName); - this.cliService = service; - currentServerContext = new ThreadLocal(); - serverEventHandler = new TServerEventHandler() { - @Override - public ServerContext createContext( - TProtocol input, TProtocol output) { - return new ThriftCLIServerContext(); - } - - @Override - public void deleteContext(ServerContext serverContext, - TProtocol input, TProtocol output) { - ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; - SessionHandle sessionHandle = context.getSessionHandle(); - if (sessionHandle != null) { - LOG.info("Session disconnected without closing properly, close it now"); - try { - cliService.closeSession(sessionHandle); - } catch (HiveSQLException e) { - LOG.warn("Failed to close session: " + e, e); - } - } - } - - @Override - public void preServe() { - } - - @Override - public void processContext(ServerContext serverContext, - TTransport input, TTransport output) { - currentServerContext.set(serverContext); - } - }; - } - - @Override - public synchronized void init(HiveConf hiveConf) { - this.hiveConf = hiveConf; - // Initialize common server configs needed in both binary & http modes - String portString; - hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); - if (hiveHost == null) { - hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); - } - try { - if (hiveHost != null && !hiveHost.isEmpty()) { - serverIPAddress = InetAddress.getByName(hiveHost); - } else { - serverIPAddress = InetAddress.getLocalHost(); - } - } catch (UnknownHostException e) { - throw new ServiceException(e); - } - // HTTP mode - if (HiveServer2.isHTTPTransportMode(hiveConf)) { - workerKeepAliveTime = - hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME, - TimeUnit.SECONDS); - portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT"); - if (portString != null) { - portNum = Integer.valueOf(portString); - } else { - portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT); - } - } - // Binary mode - else { - workerKeepAliveTime = - hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME, TimeUnit.SECONDS); - portString = System.getenv("HIVE_SERVER2_THRIFT_PORT"); - if (portString != null) { - portNum = Integer.valueOf(portString); - } else { - portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT); - } - } - minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS); - maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS); - super.init(hiveConf); - } - - @Override - public synchronized void start() { - super.start(); - if (!isStarted && !isEmbedded) { - initializeServer(); - new Thread(this).start(); - isStarted = true; - } - } - - @Override - public synchronized void stop() { - if (isStarted && !isEmbedded) { - if(server != null) { - server.stop(); - LOG.info("Thrift server has stopped"); - } - if((httpServer != null) && httpServer.isStarted()) { - try { - httpServer.stop(); - LOG.info("Http server has stopped"); - } catch (Exception e) { - LOG.error("Error stopping Http server: ", e); - } - } - isStarted = false; - } - super.stop(); - } - - public int getPortNumber() { - return portNum; - } - - public InetAddress getServerIPAddress() { - return serverIPAddress; - } - - @Override - public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) - throws TException { - TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - @Override - public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) - throws TException { - TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - @Override - public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) - throws TException { - TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); - resp.setStatus(notSupportTokenErrorStatus()); - return resp; - } - - private TStatus notSupportTokenErrorStatus() { - TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); - errorStatus.setErrorMessage("Delegation token is not supported"); - return errorStatus; - } - - @Override - public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { - LOG.info("Client protocol version: " + req.getClient_protocol()); - TOpenSessionResp resp = new TOpenSessionResp(); - try { - SessionHandle sessionHandle = getSessionHandle(req, resp); - resp.setSessionHandle(sessionHandle.toTSessionHandle()); - // TODO: set real configuration map - resp.setConfiguration(new HashMap()); - resp.setStatus(OK_STATUS); - ThriftCLIServerContext context = - (ThriftCLIServerContext)currentServerContext.get(); - if (context != null) { - context.setSessionHandle(sessionHandle); - } - } catch (Exception e) { - LOG.warn("Error opening session: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - private String getIpAddress() { - String clientIpAddress; - // Http transport mode. - // We set the thread local ip address, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - clientIpAddress = SessionManager.getIpAddress(); - } - else { - // Kerberos - if (isKerberosAuthMode()) { - clientIpAddress = hiveAuthFactory.getIpAddress(); - } - // Except kerberos, NOSASL - else { - clientIpAddress = TSetIpAddressProcessor.getUserIpAddress(); - } - } - LOG.debug("Client's IP Address: " + clientIpAddress); - return clientIpAddress; - } - - /** - * Returns the effective username. - * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user - * 2. If hive.server2.allow.user.substitution = true: the username of the end user, - * that the connecting user is trying to proxy for. - * This includes a check whether the connecting user is allowed to proxy for the end user. - * @param req - * @return - * @throws HiveSQLException - */ - private String getUserName(TOpenSessionReq req) throws HiveSQLException { - String userName = null; - // Kerberos - if (isKerberosAuthMode()) { - userName = hiveAuthFactory.getRemoteUser(); - } - // Except kerberos, NOSASL - if (userName == null) { - userName = TSetIpAddressProcessor.getUserName(); - } - // Http transport mode. - // We set the thread local username, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - userName = SessionManager.getUserName(); - } - if (userName == null) { - userName = req.getUsername(); - } - - userName = getShortName(userName); - String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress()); - LOG.debug("Client's username: " + effectiveClientUser); - return effectiveClientUser; - } - - private String getShortName(String userName) { - String ret = null; - if (userName != null) { - int indexOfDomainMatch = ServiceUtils.indexOfDomainMatch(userName); - ret = (indexOfDomainMatch <= 0) ? userName : - userName.substring(0, indexOfDomainMatch); - } - - return ret; - } - - /** - * Create a session handle - * @param req - * @param res - * @return - * @throws HiveSQLException - * @throws LoginException - * @throws IOException - */ - SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) - throws HiveSQLException, LoginException, IOException { - String userName = getUserName(req); - String ipAddress = getIpAddress(); - TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, - req.getClient_protocol()); - res.setServerProtocolVersion(protocol); - SessionHandle sessionHandle; - if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - (userName != null)) { - String delegationTokenStr = getDelegationToken(userName); - sessionHandle = cliService.openSessionWithImpersonation(protocol, userName, - req.getPassword(), ipAddress, req.getConfiguration(), delegationTokenStr); - } else { - sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), - ipAddress, req.getConfiguration()); - } - return sessionHandle; - } - - - private String getDelegationToken(String userName) - throws HiveSQLException, LoginException, IOException { - if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) - .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString())) { - return null; - } - try { - return cliService.getDelegationTokenFromMetaStore(userName); - } catch (UnsupportedOperationException e) { - // The delegation token is not applicable in the given deployment mode - } - return null; - } - - private TProtocolVersion getMinVersion(TProtocolVersion... versions) { - TProtocolVersion[] values = TProtocolVersion.values(); - int current = values[values.length - 1].getValue(); - for (TProtocolVersion version : versions) { - if (current > version.getValue()) { - current = version.getValue(); - } - } - for (TProtocolVersion version : values) { - if (version.getValue() == current) { - return version; - } - } - throw new IllegalArgumentException("never"); - } - - @Override - public TCloseSessionResp CloseSession(TCloseSessionReq req) throws TException { - TCloseSessionResp resp = new TCloseSessionResp(); - try { - SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); - cliService.closeSession(sessionHandle); - resp.setStatus(OK_STATUS); - ThriftCLIServerContext context = - (ThriftCLIServerContext)currentServerContext.get(); - if (context != null) { - context.setSessionHandle(null); - } - } catch (Exception e) { - LOG.warn("Error closing session: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetInfoResp GetInfo(TGetInfoReq req) throws TException { - TGetInfoResp resp = new TGetInfoResp(); - try { - GetInfoValue getInfoValue = - cliService.getInfo(new SessionHandle(req.getSessionHandle()), - GetInfoType.getGetInfoType(req.getInfoType())); - resp.setInfoValue(getInfoValue.toTGetInfoValue()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting info: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws TException { - TExecuteStatementResp resp = new TExecuteStatementResp(); - try { - SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); - String statement = req.getStatement(); - Map confOverlay = req.getConfOverlay(); - Boolean runAsync = req.isRunAsync(); - OperationHandle operationHandle = runAsync ? - cliService.executeStatementAsync(sessionHandle, statement, confOverlay) - : cliService.executeStatement(sessionHandle, statement, confOverlay); - resp.setOperationHandle(operationHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error executing statement: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws TException { - TGetTypeInfoResp resp = new TGetTypeInfoResp(); - try { - OperationHandle operationHandle = cliService.getTypeInfo(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(operationHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting type info: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws TException { - TGetCatalogsResp resp = new TGetCatalogsResp(); - try { - OperationHandle opHandle = cliService.getCatalogs(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting catalogs: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws TException { - TGetSchemasResp resp = new TGetSchemasResp(); - try { - OperationHandle opHandle = cliService.getSchemas( - new SessionHandle(req.getSessionHandle()), req.getCatalogName(), req.getSchemaName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting schemas: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTablesResp GetTables(TGetTablesReq req) throws TException { - TGetTablesResp resp = new TGetTablesResp(); - try { - OperationHandle opHandle = cliService - .getTables(new SessionHandle(req.getSessionHandle()), req.getCatalogName(), - req.getSchemaName(), req.getTableName(), req.getTableTypes()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting tables: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws TException { - TGetTableTypesResp resp = new TGetTableTypesResp(); - try { - OperationHandle opHandle = cliService.getTableTypes(new SessionHandle(req.getSessionHandle())); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting table types: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetColumnsResp GetColumns(TGetColumnsReq req) throws TException { - TGetColumnsResp resp = new TGetColumnsResp(); - try { - OperationHandle opHandle = cliService.getColumns( - new SessionHandle(req.getSessionHandle()), - req.getCatalogName(), - req.getSchemaName(), - req.getTableName(), - req.getColumnName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting columns: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws TException { - TGetFunctionsResp resp = new TGetFunctionsResp(); - try { - OperationHandle opHandle = cliService.getFunctions( - new SessionHandle(req.getSessionHandle()), req.getCatalogName(), - req.getSchemaName(), req.getFunctionName()); - resp.setOperationHandle(opHandle.toTOperationHandle()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting functions: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws TException { - TGetOperationStatusResp resp = new TGetOperationStatusResp(); - try { - OperationStatus operationStatus = cliService.getOperationStatus( - new OperationHandle(req.getOperationHandle())); - resp.setOperationState(operationStatus.getState().toTOperationState()); - HiveSQLException opException = operationStatus.getOperationException(); - if (opException != null) { - resp.setSqlState(opException.getSQLState()); - resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(org.apache.hadoop.util.StringUtils - .stringifyException(opException)); - } - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting operation status: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws TException { - TCancelOperationResp resp = new TCancelOperationResp(); - try { - cliService.cancelOperation(new OperationHandle(req.getOperationHandle())); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error cancelling operation: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws TException { - TCloseOperationResp resp = new TCloseOperationResp(); - try { - cliService.closeOperation(new OperationHandle(req.getOperationHandle())); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error closing operation: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) - throws TException { - TGetResultSetMetadataResp resp = new TGetResultSetMetadataResp(); - try { - TableSchema schema = cliService.getResultSetMetadata(new OperationHandle(req.getOperationHandle())); - resp.setSchema(schema.toTTableSchema()); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error getting result set metadata: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - @Override - public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { - TFetchResultsResp resp = new TFetchResultsResp(); - try { - RowSet rowSet = cliService.fetchResults( - new OperationHandle(req.getOperationHandle()), - FetchOrientation.getFetchOrientation(req.getOrientation()), - req.getMaxRows(), - FetchType.getFetchType(req.getFetchType())); - resp.setResults(rowSet.toTRowSet()); - resp.setHasMoreRows(false); - resp.setStatus(OK_STATUS); - } catch (Exception e) { - LOG.warn("Error fetching results: ", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - return resp; - } - - protected abstract void initializeServer(); - - @Override - public abstract void run(); - - /** - * If the proxy user name is provided then check privileges to substitute the user. - * @param realUser - * @param sessionConf - * @param ipAddress - * @return - * @throws HiveSQLException - */ - private String getProxyUser(String realUser, Map sessionConf, - String ipAddress) throws HiveSQLException { - String proxyUser = null; - // Http transport mode. - // We set the thread local proxy username, in ThriftHttpServlet. - if (cliService.getHiveConf().getVar( - ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { - proxyUser = SessionManager.getProxyUserName(); - LOG.debug("Proxy user from query string: " + proxyUser); - } - - if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) { - String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER); - LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody); - proxyUser = proxyUserFromThriftBody; - } - - if (proxyUser == null) { - return realUser; - } - - // check whether substitution is allowed - if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) { - throw new HiveSQLException("Proxy user substitution is not allowed"); - } - - // If there's no authentication, then directly substitute the user - if (HiveAuthFactory.AuthTypes.NONE.toString() - .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { - return proxyUser; - } - - // Verify proxy user privilege of the realUser for the proxyUser - HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf); - LOG.debug("Verified proxy user: " + proxyUser); - return proxyUser; - } - - private boolean isKerberosAuthMode() { - return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) - .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java deleted file mode 100644 index 1af45398b895c..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java +++ /dev/null @@ -1,440 +0,0 @@ -/** - * 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.hive.service.cli.thrift; - -import java.util.List; -import java.util.Map; - -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.*; -import org.apache.thrift.TException; - -/** - * ThriftCLIServiceClient. - * - */ -public class ThriftCLIServiceClient extends CLIServiceClient { - private final TCLIService.Iface cliService; - - public ThriftCLIServiceClient(TCLIService.Iface cliService) { - this.cliService = cliService; - } - - public void checkStatus(TStatus status) throws HiveSQLException { - if (TStatusCode.ERROR_STATUS.equals(status.getStatusCode())) { - throw new HiveSQLException(status); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) - */ - @Override - public SessionHandle openSession(String username, String password, - Map configuration) - throws HiveSQLException { - try { - TOpenSessionReq req = new TOpenSessionReq(); - req.setUsername(username); - req.setPassword(password); - req.setConfiguration(configuration); - TOpenSessionResp resp = cliService.OpenSession(req); - checkStatus(resp.getStatus()); - return new SessionHandle(resp.getSessionHandle(), resp.getServerProtocolVersion()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public SessionHandle openSessionWithImpersonation(String username, String password, - Map configuration, String delegationToken) throws HiveSQLException { - throw new HiveSQLException("open with impersonation operation is not supported in the client"); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { - try { - TCloseSessionReq req = new TCloseSessionReq(sessionHandle.toTSessionHandle()); - TCloseSessionResp resp = cliService.CloseSession(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) - */ - @Override - public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) - throws HiveSQLException { - try { - // FIXME extract the right info type - TGetInfoReq req = new TGetInfoReq(sessionHandle.toTSessionHandle(), infoType.toTGetInfoType()); - TGetInfoResp resp = cliService.GetInfo(req); - checkStatus(resp.getStatus()); - return new GetInfoValue(resp.getInfoValue()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(sessionHandle, statement, confOverlay, false); - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) - */ - @Override - public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, - Map confOverlay) - throws HiveSQLException { - return executeStatementInternal(sessionHandle, statement, confOverlay, true); - } - - private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement, - Map confOverlay, boolean isAsync) - throws HiveSQLException { - try { - TExecuteStatementReq req = - new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement); - req.setConfOverlay(confOverlay); - req.setRunAsync(isAsync); - TExecuteStatementResp resp = cliService.ExecuteStatement(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetTypeInfoReq req = new TGetTypeInfoReq(sessionHandle.toTSessionHandle()); - TGetTypeInfoResp resp = cliService.GetTypeInfo(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetCatalogsReq req = new TGetCatalogsReq(sessionHandle.toTSessionHandle()); - TGetCatalogsResp resp = cliService.GetCatalogs(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) - */ - @Override - public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, - String schemaName) - throws HiveSQLException { - try { - TGetSchemasReq req = new TGetSchemasReq(sessionHandle.toTSessionHandle()); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - TGetSchemasResp resp = cliService.GetSchemas(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) - */ - @Override - public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, - String schemaName, String tableName, List tableTypes) - throws HiveSQLException { - try { - TGetTablesReq req = new TGetTablesReq(sessionHandle.toTSessionHandle()); - req.setTableName(tableName); - req.setTableTypes(tableTypes); - req.setSchemaName(schemaName); - TGetTablesResp resp = cliService.GetTables(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { - try { - TGetTableTypesReq req = new TGetTableTypesReq(sessionHandle.toTSessionHandle()); - TGetTableTypesResp resp = cliService.GetTableTypes(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getColumns(SessionHandle sessionHandle, - String catalogName, String schemaName, String tableName, String columnName) - throws HiveSQLException { - try { - TGetColumnsReq req = new TGetColumnsReq(); - req.setSessionHandle(sessionHandle.toTSessionHandle()); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - req.setTableName(tableName); - req.setColumnName(columnName); - TGetColumnsResp resp = cliService.GetColumns(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) - */ - @Override - public OperationHandle getFunctions(SessionHandle sessionHandle, - String catalogName, String schemaName, String functionName) throws HiveSQLException { - try { - TGetFunctionsReq req = new TGetFunctionsReq(sessionHandle.toTSessionHandle(), functionName); - req.setCatalogName(catalogName); - req.setSchemaName(schemaName); - TGetFunctionsResp resp = cliService.GetFunctions(req); - checkStatus(resp.getStatus()); - TProtocolVersion protocol = sessionHandle.getProtocolVersion(); - return new OperationHandle(resp.getOperationHandle(), protocol); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { - try { - TGetOperationStatusReq req = new TGetOperationStatusReq(opHandle.toTOperationHandle()); - TGetOperationStatusResp resp = cliService.GetOperationStatus(req); - // Checks the status of the RPC call, throws an exception in case of error - checkStatus(resp.getStatus()); - OperationState opState = OperationState.getOperationState(resp.getOperationState()); - HiveSQLException opException = null; - if (opState == OperationState.ERROR) { - opException = new HiveSQLException(resp.getErrorMessage(), resp.getSqlState(), resp.getErrorCode()); - } - return new OperationStatus(opState, opException); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { - try { - TCancelOperationReq req = new TCancelOperationReq(opHandle.toTOperationHandle()); - TCancelOperationResp resp = cliService.CancelOperation(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public void closeOperation(OperationHandle opHandle) - throws HiveSQLException { - try { - TCloseOperationReq req = new TCloseOperationReq(opHandle.toTOperationHandle()); - TCloseOperationResp resp = cliService.CloseOperation(req); - checkStatus(resp.getStatus()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public TableSchema getResultSetMetadata(OperationHandle opHandle) - throws HiveSQLException { - try { - TGetResultSetMetadataReq req = new TGetResultSetMetadataReq(opHandle.toTOperationHandle()); - TGetResultSetMetadataResp resp = cliService.GetResultSetMetadata(req); - checkStatus(resp.getStatus()); - return new TableSchema(resp.getSchema()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - @Override - public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, - FetchType fetchType) throws HiveSQLException { - try { - TFetchResultsReq req = new TFetchResultsReq(); - req.setOperationHandle(opHandle.toTOperationHandle()); - req.setOrientation(orientation.toTFetchOrientation()); - req.setMaxRows(maxRows); - req.setFetchType(fetchType.toTFetchType()); - TFetchResultsResp resp = cliService.FetchResults(req); - checkStatus(resp.getStatus()); - return RowSetFactory.create(resp.getResults(), opHandle.getProtocolVersion()); - } catch (HiveSQLException e) { - throw e; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - /* (non-Javadoc) - * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) - */ - @Override - public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { - // TODO: set the correct default fetch size - return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000, FetchType.QUERY_OUTPUT); - } - - @Override - public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String owner, String renewer) throws HiveSQLException { - TGetDelegationTokenReq req = new TGetDelegationTokenReq( - sessionHandle.toTSessionHandle(), owner, renewer); - try { - TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req); - checkStatus(tokenResp.getStatus()); - return tokenResp.getDelegationToken(); - } catch (Exception e) { - throw new HiveSQLException(e); - } - } - - @Override - public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq( - sessionHandle.toTSessionHandle(), tokenStr); - try { - TCancelDelegationTokenResp cancelResp = - cliService.CancelDelegationToken(cancelReq); - checkStatus(cancelResp.getStatus()); - return; - } catch (TException e) { - throw new HiveSQLException(e); - } - } - - @Override - public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, - String tokenStr) throws HiveSQLException { - TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq( - sessionHandle.toTSessionHandle(), tokenStr); - try { - TRenewDelegationTokenResp renewResp = - cliService.RenewDelegationToken(cancelReq); - checkStatus(renewResp.getStatus()); - return; - } catch (Exception e) { - throw new HiveSQLException(e); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java deleted file mode 100644 index bd64c777c1d76..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ /dev/null @@ -1,194 +0,0 @@ -/** - * 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.hive.service.cli.thrift; - -import java.util.Arrays; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.thrift.TCLIService.Iface; -import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; -import org.apache.thrift.TProcessor; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.protocol.TProtocolFactory; -import org.apache.thrift.server.TServlet; -import org.eclipse.jetty.server.AbstractConnectionFactory; -import org.eclipse.jetty.server.ConnectionFactory; -import org.eclipse.jetty.server.HttpConnectionFactory; -import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.util.ssl.SslContextFactory; -import org.eclipse.jetty.util.thread.ExecutorThreadPool; -import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; - - -public class ThriftHttpCLIService extends ThriftCLIService { - - public ThriftHttpCLIService(CLIService cliService) { - super(cliService, ThriftHttpCLIService.class.getSimpleName()); - } - - @Override - protected void initializeServer() { - try { - // Server thread pool - // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests - String threadPoolName = "HiveServer2-HttpHandler-Pool"; - ThreadPoolExecutor executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, - workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), - new ThreadFactoryWithGarbageCleanup(threadPoolName)); - ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); - - // HTTP Server - httpServer = new org.eclipse.jetty.server.Server(threadPool); - - // Connector configs - - ConnectionFactory[] connectionFactories; - boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); - String schemeName = useSsl ? "https" : "http"; - // Change connector if SSL is used - if (useSsl) { - String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); - String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); - if (keyStorePath.isEmpty()) { - throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname - + " Not configured for SSL connection"); - } - SslContextFactory sslContextFactory = new SslContextFactory.Server(); - String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); - LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); - sslContextFactory.addExcludeProtocols(excludedProtocols); - LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = " + - Arrays.toString(sslContextFactory.getExcludeProtocols())); - sslContextFactory.setKeyStorePath(keyStorePath); - sslContextFactory.setKeyStorePassword(keyStorePassword); - connectionFactories = AbstractConnectionFactory.getFactories( - sslContextFactory, new HttpConnectionFactory()); - } else { - connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; - } - ServerConnector connector = new ServerConnector( - httpServer, - null, - // Call this full constructor to set this, which forces daemon threads: - new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), - null, - -1, - -1, - connectionFactories); - - connector.setPort(portNum); - // Linux:yes, Windows:no - connector.setReuseAddress(!Shell.WINDOWS); - int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, - TimeUnit.MILLISECONDS); - connector.setIdleTimeout(maxIdleTime); - - httpServer.addConnector(connector); - - // Thrift configs - hiveAuthFactory = new HiveAuthFactory(hiveConf); - TProcessor processor = new TCLIService.Processor(this); - TProtocolFactory protocolFactory = new TBinaryProtocol.Factory(); - // Set during the init phase of HiveServer2 if auth mode is kerberos - // UGI for the hive/_HOST (kerberos) principal - UserGroupInformation serviceUGI = cliService.getServiceUGI(); - // UGI for the http/_HOST (SPNego) principal - UserGroupInformation httpUGI = cliService.getHttpUGI(); - String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION); - TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, authType, - serviceUGI, httpUGI); - - // Context handler - final ServletContextHandler context = new ServletContextHandler( - ServletContextHandler.SESSIONS); - context.setContextPath("/"); - String httpPath = getHttpPath(hiveConf - .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); - httpServer.setHandler(context); - context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); - - // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc. - // Finally, start the server - httpServer.start(); - // In case HIVE_SERVER2_THRIFT_HTTP_PORT or hive.server2.thrift.http.port is configured with - // 0 which represents any free port, we should set it to the actual one - portNum = connector.getLocalPort(); - String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName - + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." - + maxWorkerThreads + " worker threads"; - LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { - httpServer.join(); - } catch (Throwable t) { - LOG.fatal( - "Error starting HiveServer2: could not start " - + ThriftHttpCLIService.class.getSimpleName(), t); - System.exit(-1); - } - } - - /** - * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. - * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" - * @param httpPath - * @return - */ - private String getHttpPath(String httpPath) { - if(httpPath == null || httpPath.equals("")) { - httpPath = "/*"; - } - else { - if(!httpPath.startsWith("/")) { - httpPath = "/" + httpPath; - } - if(httpPath.endsWith("/")) { - httpPath = httpPath + "*"; - } - if(!httpPath.endsWith("/*")) { - httpPath = httpPath + "/*"; - } - } - return httpPath; - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java deleted file mode 100644 index e15d2d0566d2b..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ /dev/null @@ -1,545 +0,0 @@ -/** - * 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.hive.service.cli.thrift; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.security.PrivilegedExceptionAction; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import javax.servlet.ServletException; -import javax.servlet.http.Cookie; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import javax.ws.rs.core.NewCookie; - -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.codec.binary.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.service.auth.AuthenticationProviderFactory; -import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; -import org.apache.hive.service.auth.HiveAuthFactory; -import org.apache.hive.service.auth.HttpAuthUtils; -import org.apache.hive.service.auth.HttpAuthenticationException; -import org.apache.hive.service.auth.PasswdAuthenticationProvider; -import org.apache.hive.service.cli.session.SessionManager; -import org.apache.hive.service.CookieSigner; -import org.apache.thrift.TProcessor; -import org.apache.thrift.protocol.TProtocolFactory; -import org.apache.thrift.server.TServlet; -import org.ietf.jgss.GSSContext; -import org.ietf.jgss.GSSCredential; -import org.ietf.jgss.GSSException; -import org.ietf.jgss.GSSManager; -import org.ietf.jgss.GSSName; -import org.ietf.jgss.Oid; - -/** - * - * ThriftHttpServlet - * - */ -public class ThriftHttpServlet extends TServlet { - - private static final long serialVersionUID = 1L; - public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName()); - private final String authType; - private final UserGroupInformation serviceUGI; - private final UserGroupInformation httpUGI; - private HiveConf hiveConf = new HiveConf(); - - // Class members for cookie based authentication. - private CookieSigner signer; - public static final String AUTH_COOKIE = "hive.server2.auth"; - private static final Random RAN = new Random(); - private boolean isCookieAuthEnabled; - private String cookieDomain; - private String cookiePath; - private int cookieMaxAge; - private boolean isCookieSecure; - private boolean isHttpOnlyCookie; - - public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, - String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI) { - super(processor, protocolFactory); - this.authType = authType; - this.serviceUGI = serviceUGI; - this.httpUGI = httpUGI; - this.isCookieAuthEnabled = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED); - // Initialize the cookie based authentication related variables. - if (isCookieAuthEnabled) { - // Generate the signer with secret. - String secret = Long.toString(RAN.nextLong()); - LOG.debug("Using the random number as the secret for cookie generation " + secret); - this.signer = new CookieSigner(secret.getBytes()); - this.cookieMaxAge = (int) hiveConf.getTimeVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, TimeUnit.SECONDS); - this.cookieDomain = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_DOMAIN); - this.cookiePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_PATH); - this.isCookieSecure = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_SECURE); - this.isHttpOnlyCookie = hiveConf.getBoolVar( - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_HTTPONLY); - } - } - - @Override - protected void doPost(HttpServletRequest request, HttpServletResponse response) - throws ServletException, IOException { - String clientUserName = null; - String clientIpAddress; - boolean requireNewCookie = false; - - try { - // If the cookie based authentication is already enabled, parse the - // request and validate the request cookies. - if (isCookieAuthEnabled) { - clientUserName = validateCookie(request); - requireNewCookie = (clientUserName == null); - if (requireNewCookie) { - LOG.info("Could not validate cookie sent, will try to generate a new cookie"); - } - } - // If the cookie based authentication is not enabled or the request does - // not have a valid cookie, use the kerberos or password based authentication - // depending on the server setup. - if (clientUserName == null) { - // For a kerberos setup - if (isKerberosAuthMode(authType)) { - clientUserName = doKerberosAuth(request); - } - // For password based authentication - else { - clientUserName = doPasswdAuth(request, authType); - } - } - LOG.debug("Client username: " + clientUserName); - - // Set the thread local username to be used for doAs if true - SessionManager.setUserName(clientUserName); - - // find proxy user if any from query param - String doAsQueryParam = getDoAsQueryParam(request.getQueryString()); - if (doAsQueryParam != null) { - SessionManager.setProxyUserName(doAsQueryParam); - } - - clientIpAddress = request.getRemoteAddr(); - LOG.debug("Client IP Address: " + clientIpAddress); - // Set the thread local ip address - SessionManager.setIpAddress(clientIpAddress); - // Generate new cookie and add it to the response - if (requireNewCookie && - !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { - String cookieToken = HttpAuthUtils.createCookieToken(clientUserName); - Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken)); - - if (isHttpOnlyCookie) { - response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)); - } else { - response.addCookie(hs2Cookie); - } - LOG.info("Cookie added for clientUserName " + clientUserName); - } - super.doPost(request, response); - } - catch (HttpAuthenticationException e) { - LOG.error("Error: ", e); - // Send a 401 to the client - response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); - if(isKerberosAuthMode(authType)) { - response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); - } - response.getWriter().println("Authentication Error: " + e.getMessage()); - } - finally { - // Clear the thread locals - SessionManager.clearUserName(); - SessionManager.clearIpAddress(); - SessionManager.clearProxyUserName(); - } - } - - /** - * Retrieves the client name from cookieString. If the cookie does not - * correspond to a valid client, the function returns null. - * @param cookies HTTP Request cookies. - * @return Client Username if cookieString has a HS2 Generated cookie that is currently valid. - * Else, returns null. - */ - private String getClientNameFromCookie(Cookie[] cookies) { - // Current Cookie Name, Current Cookie Value - String currName, currValue; - - // Following is the main loop which iterates through all the cookies send by the client. - // The HS2 generated cookies are of the format hive.server2.auth= - // A cookie which is identified as a hiveserver2 generated cookie is validated - // by calling signer.verifyAndExtract(). If the validation passes, send the - // username for which the cookie is validated to the caller. If no client side - // cookie passes the validation, return null to the caller. - for (Cookie currCookie : cookies) { - // Get the cookie name - currName = currCookie.getName(); - if (!currName.equals(AUTH_COOKIE)) { - // Not a HS2 generated cookie, continue. - continue; - } - // If we reached here, we have match for HS2 generated cookie - currValue = currCookie.getValue(); - // Validate the value. - currValue = signer.verifyAndExtract(currValue); - // Retrieve the user name, do the final validation step. - if (currValue != null) { - String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); - - if (userName == null) { - LOG.warn("Invalid cookie token " + currValue); - continue; - } - //We have found a valid cookie in the client request. - if (LOG.isDebugEnabled()) { - LOG.debug("Validated the cookie for user " + userName); - } - return userName; - } - } - // No valid HS2 generated cookies found, return null - return null; - } - - /** - * Convert cookie array to human readable cookie string - * @param cookies Cookie Array - * @return String containing all the cookies separated by a newline character. - * Each cookie is of the format [key]=[value] - */ - private String toCookieStr(Cookie[] cookies) { - String cookieStr = ""; - - for (Cookie c : cookies) { - cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; - } - return cookieStr; - } - - /** - * Validate the request cookie. This function iterates over the request cookie headers - * and finds a cookie that represents a valid client/server session. If it finds one, it - * returns the client name associated with the session. Else, it returns null. - * @param request The HTTP Servlet Request send by the client - * @return Client Username if the request has valid HS2 cookie, else returns null - * @throws UnsupportedEncodingException - */ - private String validateCookie(HttpServletRequest request) throws UnsupportedEncodingException { - // Find all the valid cookies associated with the request. - Cookie[] cookies = request.getCookies(); - - if (cookies == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("No valid cookies associated with the request " + request); - } - return null; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Received cookies: " + toCookieStr(cookies)); - } - return getClientNameFromCookie(cookies); - } - - /** - * Generate a server side cookie given the cookie value as the input. - * @param str Input string token. - * @return The generated cookie. - * @throws UnsupportedEncodingException - */ - private Cookie createCookie(String str) throws UnsupportedEncodingException { - if (LOG.isDebugEnabled()) { - LOG.debug("Cookie name = " + AUTH_COOKIE + " value = " + str); - } - Cookie cookie = new Cookie(AUTH_COOKIE, str); - - cookie.setMaxAge(cookieMaxAge); - if (cookieDomain != null) { - cookie.setDomain(cookieDomain); - } - if (cookiePath != null) { - cookie.setPath(cookiePath); - } - cookie.setSecure(isCookieSecure); - return cookie; - } - - /** - * Generate httponly cookie from HS2 cookie - * @param cookie HS2 generated cookie - * @return The httponly cookie - */ - private static String getHttpOnlyCookieHeader(Cookie cookie) { - NewCookie newCookie = new NewCookie(cookie.getName(), cookie.getValue(), - cookie.getPath(), cookie.getDomain(), cookie.getVersion(), - cookie.getComment(), cookie.getMaxAge(), cookie.getSecure()); - return newCookie + "; HttpOnly"; - } - - /** - * Do the LDAP/PAM authentication - * @param request - * @param authType - * @throws HttpAuthenticationException - */ - private String doPasswdAuth(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String userName = getUsername(request, authType); - // No-op when authType is NOSASL - if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { - try { - AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType); - PasswdAuthenticationProvider provider = - AuthenticationProviderFactory.getAuthenticationProvider(authMethod); - provider.Authenticate(userName, getPassword(request, authType)); - - } catch (Exception e) { - throw new HttpAuthenticationException(e); - } - } - return userName; - } - - /** - * Do the GSS-API kerberos authentication. - * We already have a logged in subject in the form of serviceUGI, - * which GSS-API will extract information from. - * In case of a SPNego request we use the httpUGI, - * for the authenticating service tickets. - * @param request - * @return - * @throws HttpAuthenticationException - */ - private String doKerberosAuth(HttpServletRequest request) - throws HttpAuthenticationException { - // Try authenticating with the http/_HOST principal - if (httpUGI != null) { - try { - return httpUGI.doAs(new HttpKerberosServerAction(request, httpUGI)); - } catch (Exception e) { - LOG.info("Failed to authenticate with http/_HOST kerberos principal, " + - "trying with hive/_HOST kerberos principal"); - } - } - // Now try with hive/_HOST principal - try { - return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI)); - } catch (Exception e) { - LOG.error("Failed to authenticate with hive/_HOST kerberos principal"); - throw new HttpAuthenticationException(e); - } - - } - - class HttpKerberosServerAction implements PrivilegedExceptionAction { - HttpServletRequest request; - UserGroupInformation serviceUGI; - - HttpKerberosServerAction(HttpServletRequest request, - UserGroupInformation serviceUGI) { - this.request = request; - this.serviceUGI = serviceUGI; - } - - @Override - public String run() throws HttpAuthenticationException { - // Get own Kerberos credentials for accepting connection - GSSManager manager = GSSManager.getInstance(); - GSSContext gssContext = null; - String serverPrincipal = getPrincipalWithoutRealm( - serviceUGI.getUserName()); - try { - // This Oid for Kerberos GSS-API mechanism. - Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2"); - // Oid for SPNego GSS-API mechanism. - Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2"); - // Oid for kerberos principal name - Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); - - // GSS name for server - GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); - - // GSS credentials for server - GSSCredential serverCreds = manager.createCredential(serverName, - GSSCredential.DEFAULT_LIFETIME, - new Oid[]{kerberosMechOid, spnegoMechOid}, - GSSCredential.ACCEPT_ONLY); - - // Create a GSS context - gssContext = manager.createContext(serverCreds); - // Get service ticket from the authorization header - String serviceTicketBase64 = getAuthHeader(request, authType); - byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes()); - gssContext.acceptSecContext(inToken, 0, inToken.length); - // Authenticate or deny based on its context completion - if (!gssContext.isEstablished()) { - throw new HttpAuthenticationException("Kerberos authentication failed: " + - "unable to establish context with the service ticket " + - "provided by the client."); - } - else { - return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString()); - } - } - catch (GSSException e) { - throw new HttpAuthenticationException("Kerberos authentication failed: ", e); - } - finally { - if (gssContext != null) { - try { - gssContext.dispose(); - } catch (GSSException e) { - // No-op - } - } - } - } - - private String getPrincipalWithoutRealm(String fullPrincipal) - throws HttpAuthenticationException { - KerberosNameShim fullKerberosName; - try { - fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); - } catch (IOException e) { - throw new HttpAuthenticationException(e); - } - String serviceName = fullKerberosName.getServiceName(); - String hostName = fullKerberosName.getHostName(); - String principalWithoutRealm = serviceName; - if (hostName != null) { - principalWithoutRealm = serviceName + "/" + hostName; - } - return principalWithoutRealm; - } - - private String getPrincipalWithoutRealmAndHost(String fullPrincipal) - throws HttpAuthenticationException { - KerberosNameShim fullKerberosName; - try { - fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); - return fullKerberosName.getShortName(); - } catch (IOException e) { - throw new HttpAuthenticationException(e); - } - } - } - - private String getUsername(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String[] creds = getAuthHeaderTokens(request, authType); - // Username must be present - if (creds[0] == null || creds[0].isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain username."); - } - return creds[0]; - } - - private String getPassword(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String[] creds = getAuthHeaderTokens(request, authType); - // Password must be present - if (creds[1] == null || creds[1].isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain username."); - } - return creds[1]; - } - - private String[] getAuthHeaderTokens(HttpServletRequest request, - String authType) throws HttpAuthenticationException { - String authHeaderBase64 = getAuthHeader(request, authType); - String authHeaderString = StringUtils.newStringUtf8( - Base64.decodeBase64(authHeaderBase64.getBytes())); - String[] creds = authHeaderString.split(":"); - return creds; - } - - /** - * Returns the base64 encoded auth header payload - * @param request - * @param authType - * @return - * @throws HttpAuthenticationException - */ - private String getAuthHeader(HttpServletRequest request, String authType) - throws HttpAuthenticationException { - String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION); - // Each http request must have an Authorization header - if (authHeader == null || authHeader.isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client is empty."); - } - - String authHeaderBase64String; - int beginIndex; - if (isKerberosAuthMode(authType)) { - beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length(); - } - else { - beginIndex = (HttpAuthUtils.BASIC + " ").length(); - } - authHeaderBase64String = authHeader.substring(beginIndex); - // Authorization header must have a payload - if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) { - throw new HttpAuthenticationException("Authorization header received " + - "from the client does not contain any data."); - } - return authHeaderBase64String; - } - - private boolean isKerberosAuthMode(String authType) { - return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); - } - - private static String getDoAsQueryParam(String queryString) { - if (LOG.isDebugEnabled()) { - LOG.debug("URL query string:" + queryString); - } - if (queryString == null) { - return null; - } - Map params = javax.servlet.http.HttpUtils.parseQueryString( queryString ); - Set keySet = params.keySet(); - for (String key: keySet) { - if (key.equalsIgnoreCase("doAs")) { - return params.get(key)[0]; - } - } - return null; - } - -} - - diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java deleted file mode 100644 index 95233996cbbcb..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ /dev/null @@ -1,277 +0,0 @@ -/** - * 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.hive.service.server; - -import java.util.Properties; - -import scala.runtime.AbstractFunction0; -import scala.runtime.BoxedUnit; - -import org.apache.commons.cli.GnuParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.OptionBuilder; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.common.util.HiveStringUtils; -import org.apache.hive.service.CompositeService; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService; -import org.apache.hive.service.cli.thrift.ThriftCLIService; -import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; - -import org.apache.spark.util.ShutdownHookManager; - -/** - * HiveServer2. - * - */ -public class HiveServer2 extends CompositeService { - private static final Log LOG = LogFactory.getLog(HiveServer2.class); - - private CLIService cliService; - private ThriftCLIService thriftCLIService; - - public HiveServer2() { - super(HiveServer2.class.getSimpleName()); - HiveConf.setLoadHiveServer2Config(true); - } - - @Override - public synchronized void init(HiveConf hiveConf) { - cliService = new CLIService(this); - addService(cliService); - if (isHTTPTransportMode(hiveConf)) { - thriftCLIService = new ThriftHttpCLIService(cliService); - } else { - thriftCLIService = new ThriftBinaryCLIService(cliService); - } - addService(thriftCLIService); - super.init(hiveConf); - - // Add a shutdown hook for catching SIGTERM & SIGINT - // this must be higher than the Hadoop Filesystem priority of 10, - // which the default priority is. - // The signature of the callback must match that of a scala () -> Unit - // function - ShutdownHookManager.addShutdownHook( - new AbstractFunction0() { - public BoxedUnit apply() { - try { - LOG.info("Hive Server Shutdown hook invoked"); - stop(); - } catch (Throwable e) { - LOG.warn("Ignoring Exception while stopping Hive Server from shutdown hook", - e); - } - return BoxedUnit.UNIT; - } - }); - } - - public static boolean isHTTPTransportMode(HiveConf hiveConf) { - String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE"); - if (transportMode == null) { - transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); - } - if (transportMode != null && (transportMode.equalsIgnoreCase("http"))) { - return true; - } - return false; - } - - @Override - public synchronized void start() { - super.start(); - } - - @Override - public synchronized void stop() { - LOG.info("Shutting down HiveServer2"); - super.stop(); - } - - private static void startHiveServer2() throws Throwable { - long attempts = 0, maxAttempts = 1; - while (true) { - LOG.info("Starting HiveServer2"); - HiveConf hiveConf = new HiveConf(); - maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS); - HiveServer2 server = null; - try { - server = new HiveServer2(); - server.init(hiveConf); - server.start(); - ShimLoader.getHadoopShims().startPauseMonitor(hiveConf); - break; - } catch (Throwable throwable) { - if (server != null) { - try { - server.stop(); - } catch (Throwable t) { - LOG.info("Exception caught when calling stop of HiveServer2 before retrying start", t); - } finally { - server = null; - } - } - if (++attempts >= maxAttempts) { - throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); - } else { - LOG.warn("Error starting HiveServer2 on attempt " + attempts - + ", will retry in 60 seconds", throwable); - try { - Thread.sleep(60L * 1000L); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - } - } - - public static void main(String[] args) { - HiveConf.setLoadHiveServer2Config(true); - ServerOptionsProcessor oproc = new ServerOptionsProcessor("hiveserver2"); - ServerOptionsProcessorResponse oprocResponse = oproc.parse(args); - - HiveStringUtils.startupShutdownMessage(HiveServer2.class, args, LOG); - - // Call the executor which will execute the appropriate command based on the parsed options - oprocResponse.getServerOptionsExecutor().execute(); - } - - /** - * ServerOptionsProcessor. - * Process arguments given to HiveServer2 (-hiveconf property=value) - * Set properties in System properties - * Create an appropriate response object, - * which has executor to execute the appropriate command based on the parsed options. - */ - public static class ServerOptionsProcessor { - private final Options options = new Options(); - private org.apache.commons.cli.CommandLine commandLine; - private final String serverName; - private final StringBuilder debugMessage = new StringBuilder(); - - @SuppressWarnings("static-access") - public ServerOptionsProcessor(String serverName) { - this.serverName = serverName; - // -hiveconf x=y - options.addOption(OptionBuilder - .withValueSeparator() - .hasArgs(2) - .withArgName("property=value") - .withLongOpt("hiveconf") - .withDescription("Use value for given property") - .create()); - options.addOption(new Option("H", "help", false, "Print help information")); - } - - public ServerOptionsProcessorResponse parse(String[] argv) { - try { - commandLine = new GnuParser().parse(options, argv); - // Process --hiveconf - // Get hiveconf param values and set the System property values - Properties confProps = commandLine.getOptionProperties("hiveconf"); - for (String propKey : confProps.stringPropertyNames()) { - // save logging message for log4j output latter after log4j initialize properly - debugMessage.append("Setting " + propKey + "=" + confProps.getProperty(propKey) + ";\n"); - System.setProperty(propKey, confProps.getProperty(propKey)); - } - - // Process --help - if (commandLine.hasOption('H')) { - return new ServerOptionsProcessorResponse(new HelpOptionExecutor(serverName, options)); - } - } catch (ParseException e) { - // Error out & exit - we were not able to parse the args successfully - System.err.println("Error starting HiveServer2 with given arguments: "); - System.err.println(e.getMessage()); - System.exit(-1); - } - // Default executor, when no option is specified - return new ServerOptionsProcessorResponse(new StartOptionExecutor()); - } - - StringBuilder getDebugMessage() { - return debugMessage; - } - } - - /** - * The response sent back from {@link ServerOptionsProcessor#parse(String[])} - */ - static class ServerOptionsProcessorResponse { - private final ServerOptionsExecutor serverOptionsExecutor; - - ServerOptionsProcessorResponse(ServerOptionsExecutor serverOptionsExecutor) { - this.serverOptionsExecutor = serverOptionsExecutor; - } - - ServerOptionsExecutor getServerOptionsExecutor() { - return serverOptionsExecutor; - } - } - - /** - * The executor interface for running the appropriate HiveServer2 command based on parsed options - */ - interface ServerOptionsExecutor { - void execute(); - } - - /** - * HelpOptionExecutor: executes the --help option by printing out the usage - */ - static class HelpOptionExecutor implements ServerOptionsExecutor { - private final Options options; - private final String serverName; - - HelpOptionExecutor(String serverName, Options options) { - this.options = options; - this.serverName = serverName; - } - - @Override - public void execute() { - new HelpFormatter().printHelp(serverName, options); - System.exit(0); - } - } - - /** - * StartOptionExecutor: starts HiveServer2. - * This is the default executor, when no option is specified. - */ - static class StartOptionExecutor implements ServerOptionsExecutor { - @Override - public void execute() { - try { - startHiveServer2(); - } catch (Throwable t) { - LOG.fatal("Error starting HiveServer2", t); - System.exit(-1); - } - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java deleted file mode 100644 index 8ee98103f7ef7..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * 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.hive.service.server; - -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.metastore.RawStore; - -/** - * A HiveServer2 thread used to construct new server threads. - * In particular, this thread ensures an orderly cleanup, - * when killed by its corresponding ExecutorService. - */ -public class ThreadWithGarbageCleanup extends Thread { - private static final Log LOG = LogFactory.getLog(ThreadWithGarbageCleanup.class); - - Map threadRawStoreMap = - ThreadFactoryWithGarbageCleanup.getThreadRawStoreMap(); - - public ThreadWithGarbageCleanup(Runnable runnable) { - super(runnable); - } - - /** - * Add any Thread specific garbage cleanup code here. - * Currently, it shuts down the RawStore object for this thread if it is not null. - */ - @Override - public void finalize() throws Throwable { - cleanRawStore(); - super.finalize(); - } - - private void cleanRawStore() { - Long threadId = this.getId(); - RawStore threadLocalRawStore = threadRawStoreMap.get(threadId); - if (threadLocalRawStore != null) { - LOG.debug("RawStore: " + threadLocalRawStore + ", for the thread: " + - this.getName() + " will be closed now."); - threadLocalRawStore.shutdown(); - threadRawStoreMap.remove(threadId); - } - } - - /** - * Cache the ThreadLocal RawStore object. Called from the corresponding thread. - */ - public void cacheThreadLocalRawStore() { - Long threadId = this.getId(); - RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); - if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { - LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + - this.getName() + " to threadRawStoreMap for future cleanup."); - threadRawStoreMap.put(threadId, threadLocalRawStore); - } - } -} diff --git a/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala deleted file mode 100644 index 9a28dd6a31e6e..0000000000000 --- a/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.spark.sql.hive.thriftserver - -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} -import org.apache.hive.service.cli.Type._ -import org.apache.hive.service.cli.thrift.TProtocolVersion._ - -/** - * Various utilities for hive-thriftserver used to upgrade the built-in Hive. - */ -private[thriftserver] object ThriftserverShimUtils { - - private[thriftserver] object TOperationType { - val GET_TYPE_INFO = org.apache.hive.service.cli.thrift.TOperationType.GET_TYPE_INFO - } - - private[thriftserver] type TProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion - private[thriftserver] type Client = org.apache.hive.service.cli.thrift.TCLIService.Client - private[thriftserver] type TOpenSessionReq = org.apache.hive.service.cli.thrift.TOpenSessionReq - private[thriftserver] type TGetSchemasReq = org.apache.hive.service.cli.thrift.TGetSchemasReq - private[thriftserver] type TGetTablesReq = org.apache.hive.service.cli.thrift.TGetTablesReq - private[thriftserver] type TGetColumnsReq = org.apache.hive.service.cli.thrift.TGetColumnsReq - private[thriftserver] type TGetInfoReq = org.apache.hive.service.cli.thrift.TGetInfoReq - private[thriftserver] type TExecuteStatementReq = - org.apache.hive.service.cli.thrift.TExecuteStatementReq - private[thriftserver] type THandleIdentifier = - org.apache.hive.service.cli.thrift.THandleIdentifier - private[thriftserver] type TOperationType = org.apache.hive.service.cli.thrift.TOperationType - private[thriftserver] type TOperationHandle = org.apache.hive.service.cli.thrift.TOperationHandle - - private[thriftserver] def getConsole: SessionState.LogHelper = { - val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) - new SessionState.LogHelper(LOG) - } - - private[thriftserver] def resultRowSet( - getResultSetSchema: TableSchema, - getProtocolVersion: TProtocolVersion): RowSet = { - RowSetFactory.create(getResultSetSchema, getProtocolVersion) - } - - private[thriftserver] def supportedType(): Seq[Type] = { - Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, - TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, - FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, - DATE_TYPE, TIMESTAMP_TYPE, - ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) - } - - private[thriftserver] val testedProtocolVersions = Seq( - HIVE_CLI_SERVICE_PROTOCOL_V1, - HIVE_CLI_SERVICE_PROTOCOL_V2, - HIVE_CLI_SERVICE_PROTOCOL_V3, - HIVE_CLI_SERVICE_PROTOCOL_V4, - HIVE_CLI_SERVICE_PROTOCOL_V5, - HIVE_CLI_SERVICE_PROTOCOL_V6, - HIVE_CLI_SERVICE_PROTOCOL_V7, - HIVE_CLI_SERVICE_PROTOCOL_V8) -} diff --git a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt b/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt deleted file mode 100644 index 85884a1aaf739..0000000000000 --- a/sql/hive/benchmarks/InsertIntoHiveTableBenchmark-hive1.2-results.txt +++ /dev/null @@ -1,11 +0,0 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz -insert hive table benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -INSERT INTO DYNAMIC 6812 7043 328 0.0 665204.8 1.0X -INSERT INTO HYBRID 817 852 32 0.0 79783.6 8.3X -INSERT INTO STATIC 231 246 21 0.0 22568.2 29.5X -INSERT OVERWRITE DYNAMIC 25947 26671 1024 0.0 2533910.2 0.3X -INSERT OVERWRITE HYBRID 2846 2884 54 0.0 277908.7 2.4X -INSERT OVERWRITE STATIC 232 247 26 0.0 22659.9 29.4X - diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 805bcb2bc3a60..1611a3da8a3da 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -296,8 +296,7 @@ private[hive] class HiveClientImpl( case e: NoClassDefFoundError if HiveUtils.isHive23 && e.getMessage.contains("org/apache/hadoop/hive/serde2/SerDe") => throw new ClassNotFoundException("The SerDe interface removed since Hive 2.3(HIVE-15167)." + - " Please migrate your custom SerDes to Hive 2.3 or build your own Spark with" + - " hive-1.2 profile. See HIVE-15167 for more details.", e) + " Please migrate your custom SerDes to Hive 2.3. See HIVE-15167 for more details.", e) } finally { state.getConf.setClassLoader(originalConfLoader) Thread.currentThread().setContextClassLoader(original) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala index 81eb5e2591f13..da34c54cb36a2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala @@ -28,14 +28,11 @@ import org.apache.spark.sql.hive.test.TestHive * {{{ * 1. without sbt: bin/spark-submit --class * --jars ,, - * --packages org.spark-project.hive:hive-exec:1.2.1.spark2 * - * 2. build/sbt "hive/test:runMain " -Phive-1.2 or - * build/sbt "hive/test:runMain " -Phive-2.3 + * 2. build/sbt "hive/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " * Results will be written to "benchmarks/InsertIntoHiveTableBenchmark-hive2.3-results.txt". - * 4. -Phive-1.2 does not work for JDK 11 * }}} */ object InsertIntoHiveTableBenchmark extends SqlBasedBenchmark { @@ -136,5 +133,5 @@ object InsertIntoHiveTableBenchmark extends SqlBasedBenchmark { } } - override def suffix: String = if (HiveUtils.isHive23) "-hive2.3" else "-hive1.2" + override def suffix: String = "-hive2.3" } From a0aa8f33a9420feb9228b51a3dfad2e7e86d65a5 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 6 Oct 2020 09:09:58 +0900 Subject: [PATCH 41/59] [SPARK-33069][INFRA] Skip test result report if no JUnit XML files are found ### What changes were proposed in this pull request? This PR proposes to skip test reporting ("Report test results") if there are no JUnit XML files are found. Currently, we're running and skipping the tests dynamically. For example, - if there are only changes in SparkR at the underlying commit, it only runs the SparkR tests, and skip the other tests and generate JUnit XML files for SparkR test cases. - if there are only changes in `docs` at the underlying commit, the build skips all tests except linters and do not generate any JUnit XML files. When test reporting ("Report test results") job is triggered after the main build ("Build and test ") is finished, and there are no JUnit XML files found, it reports the case as a failure. See https://github.com/apache/spark/runs/1196184007 as an example. This PR works around it by simply skipping the testing report when there are no JUnit XML files are found. Please see https://github.com/apache/spark/pull/29906#issuecomment-702525542 for more details. ### Why are the changes needed? To avoid false alarm for test results. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested in my fork. Positive case: https://github.com/HyukjinKwon/spark/runs/1208624679?check_suite_focus=true https://github.com/HyukjinKwon/spark/actions/runs/288996327 Negative case: https://github.com/HyukjinKwon/spark/runs/1208229838?check_suite_focus=true https://github.com/HyukjinKwon/spark/actions/runs/289000058 Closes #29946 from HyukjinKwon/test-junit-files. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/test_report.yml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.github/workflows/test_report.yml b/.github/workflows/test_report.yml index 93cdb86687261..060a8795b6a77 100644 --- a/.github/workflows/test_report.yml +++ b/.github/workflows/test_report.yml @@ -15,7 +15,16 @@ jobs: github_token: ${{ secrets.GITHUB_TOKEN }} workflow: ${{ github.event.workflow_run.workflow_id }} commit: ${{ github.event.workflow_run.head_commit.id }} + - name: Check if JUnit report XML files exist + run: | + if ls **/target/test-reports/*.xml > /dev/null 2>&1; then + echo '::set-output name=FILE_EXISTS::true' + else + echo '::set-output name=FILE_EXISTS::false' + fi + id: check-junit-file - name: Publish test report + if: steps.check-junit-file.outputs.FILE_EXISTS == 'true' uses: scacap/action-surefire-report@v1 with: check_name: Report test results From 9870cf9c086172a390c80f5ef23aacfe2ce3f2cf Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 6 Oct 2020 13:01:57 +0900 Subject: [PATCH 42/59] [SPARK-33067][SQL][TESTS] Add negative checks to JDBC v2 Table Catalog tests ### What changes were proposed in this pull request? Add checks for the cases when JDBC v2 Table Catalog commands fail. ### Why are the changes needed? To improve test coverage. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running `JDBCTableCatalogSuite`. Closes #29945 from MaxGekk/jdbcv2-negative-tests. Lead-authored-by: Max Gekk Co-authored-by: Hyukjin Kwon Signed-off-by: HyukjinKwon --- .../v2/jdbc/JDBCTableCatalogSuite.scala | 114 +++++++++++++++++- 1 file changed, 111 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index b308934ba03c0..bf71f90779b71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -20,7 +20,8 @@ import java.sql.{Connection, DriverManager} import java.util.Properties import org.apache.spark.SparkConf -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -63,6 +64,8 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { test("show tables") { checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + // Check not existing namespace + checkAnswer(sql("SHOW TABLES IN h2.bad_test"), Seq()) } test("drop a table and test whether the table exists") { @@ -72,6 +75,11 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "to_drop"), Row("test", "people"))) sql("DROP TABLE h2.test.to_drop") checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[NoSuchTableException] { + sql(s"DROP TABLE $table") + } + } } test("rename a table") { @@ -87,6 +95,26 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("SHOW TABLES IN h2.test"), Seq(Row("test", "dst_table"), Row("test", "people"))) } + // Rename not existing table or namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[org.h2.jdbc.JdbcSQLException] { + sql(s"ALTER TABLE $table RENAME TO test.dst_table") + } + } + // Rename to an existing table + withTable("h2.test.dst_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."dst_table" (id INTEGER)""").executeUpdate() + } + withTable("h2.test.src_table") { + withConnection { conn => + conn.prepareStatement("""CREATE TABLE "test"."src_table" (id INTEGER)""").executeUpdate() + } + intercept[org.h2.jdbc.JdbcSQLException] { + sql("ALTER TABLE h2.test.src_table RENAME TO h2.test.dst_table") + } + } + } } test("load a table") { @@ -95,6 +123,11 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("NAME", StringType) .add("ID", IntegerType) assert(t.schema === expectedSchema) + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + spark.table(s"h2.$table").schema + } + } } test("create a table") { @@ -105,6 +138,15 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"), Row("test", "new_table"))) } + withTable("h2.test.new_table") { + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + intercept[AnalysisException] { + sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") + } + } + intercept[org.h2.jdbc.JdbcSQLException] { + sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING) USING _") + } } test("alter table ... add column") { @@ -121,16 +163,38 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { t = spark.table("h2.test.alt_table") expectedSchema = expectedSchema.add("C3", DoubleType) assert(t.schema === expectedSchema) + // Add already existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C3 DOUBLE)") + } + } + // Add a column to not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table ADD COLUMNS (C4 STRING)") + } } } test("alter table ... rename column") { withTable("h2.test.alt_table") { - sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") + sql("CREATE TABLE h2.test.alt_table (ID INTEGER, C0 INTEGER) USING _") sql("ALTER TABLE h2.test.alt_table RENAME COLUMN ID TO C") val t = spark.table("h2.test.alt_table") - val expectedSchema = new StructType().add("C", IntegerType) + val expectedSchema = new StructType() + .add("C", IntegerType) + .add("C0", IntegerType) assert(t.schema === expectedSchema) + // Rename to already existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table RENAME COLUMN C TO C0") + } + } + // Rename a column in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table RENAME COLUMN ID TO C") + } } } @@ -141,6 +205,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val t = spark.table("h2.test.alt_table") val expectedSchema = new StructType().add("C2", IntegerType) assert(t.schema === expectedSchema) + // Drop not existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table DROP COLUMN bad_column") + } + } + // Drop a column to not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table DROP COLUMN C1") + } } } @@ -151,6 +225,20 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val t = spark.table("h2.test.alt_table") val expectedSchema = new StructType().add("ID", DoubleType) assert(t.schema === expectedSchema) + // Update not existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column TYPE DOUBLE") + } + // Update column to wrong type + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE bad_type") + } + } + // Update column type in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN id TYPE DOUBLE") + } } } @@ -161,6 +249,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val t = spark.table("h2.test.alt_table") val expectedSchema = new StructType().add("ID", IntegerType, nullable = true) assert(t.schema === expectedSchema) + // Update nullability of not existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column DROP NOT NULL") + } + } + // Update column nullability in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN ID DROP NOT NULL") + } } } @@ -171,6 +269,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID COMMENT 'test'") } assert(thrown.getMessage.contains("Unsupported TableChange")) + // Update comment for not existing column + intercept[AnalysisException] { + sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column COMMENT 'test'") + } + } + // Update column comments in not existing table and namespace + Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => + intercept[AnalysisException] { + sql(s"ALTER TABLE $table ALTER COLUMN ID COMMENT 'test'") + } } } } From 4adc2822a3c7b7552b436ffb61d5c134680e56b3 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 6 Oct 2020 08:32:55 +0000 Subject: [PATCH 43/59] [SPARK-33035][SQL] Updates the obsoleted entries of attribute mapping in QueryPlan#transformUpWithNewOutput ### What changes were proposed in this pull request? This PR intends to fix corner-case bugs in the `QueryPlan#transformUpWithNewOutput` that is used to propagate updated `ExprId`s in a bottom-up way. Let's say we have a rule to simply assign new `ExprId`s in a projection list like this; ``` case class TestRule extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithNewOutput { case p Project(projList, _) => val newPlan = p.copy(projectList = projList.map { _.transform { // Assigns a new `ExprId` for references case a: AttributeReference => Alias(a, a.name)() }}.asInstanceOf[Seq[NamedExpression]]) val attrMapping = p.output.zip(newPlan.output) newPlan -> attrMapping } } ``` Then, this rule is applied into a plan below; ``` (3) Project [a#5, b#6] +- (2) Project [a#5, b#6] +- (1) Project [a#5, b#6] +- LocalRelation , [a#5, b#6] ``` In the first transformation, the rule assigns new `ExprId`s in `(1) Project` (e.g., a#5 AS a#7, b#6 AS b#8). In the second transformation, the rule corrects the input references of `(2) Project` first by using attribute mapping given from `(1) Project` (a#5->a#7 and b#6->b#8) and then assigns new `ExprId`s (e.g., a#7 AS a#9, b#8 AS b#10). But, in the third transformation, the rule fails because it tries to correct the references of `(3) Project` by using incorrect attribute mapping (a#7->a#9 and b#8->b#10) even though the correct one is a#5->a#9 and b#6->b#10. To fix this issue, this PR modified the code to update the attribute mapping entries that are obsoleted by generated entries in a given rule. ### Why are the changes needed? bugfix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in `QueryPlanSuite`. Closes #29911 from maropu/QueryPlanBug. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/plans/QueryPlan.scala | 26 ++++++++++++++----- .../sql/catalyst/plans/QueryPlanSuite.scala | 26 ++++++++++++++++--- 2 files changed, 41 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index a89f055e2ac80..3e8467bab0348 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -201,11 +201,6 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case (oldAttr, _) => plan.references.contains(oldAttr) } - val (planAfterRule, newAttrMapping) = CurrentOrigin.withOrigin(origin) { - rule.applyOrElse(newPlan, (plan: PlanType) => plan -> Nil) - } - newPlan = planAfterRule - if (attrMappingForCurrentPlan.nonEmpty) { assert(!attrMappingForCurrentPlan.groupBy(_._1.exprId) .exists(_._2.map(_._2.exprId).distinct.length > 1), @@ -222,10 +217,27 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT } } - attrMapping ++= newAttrMapping.filter { + val (planAfterRule, newAttrMapping) = CurrentOrigin.withOrigin(origin) { + rule.applyOrElse(newPlan, (plan: PlanType) => plan -> Nil) + } + + val newValidAttrMapping = newAttrMapping.filter { case (a1, a2) => a1.exprId != a2.exprId } - newPlan -> attrMapping.toSeq + + // Updates the `attrMapping` entries that are obsoleted by generated entries in `rule`. + // For example, `attrMapping` has a mapping entry 'id#1 -> id#2' and `rule` + // generates a new entry 'id#2 -> id#3'. In this case, we need to update + // the corresponding old entry from 'id#1 -> id#2' to '#id#1 -> #id#3'. + val updatedAttrMap = AttributeMap(newValidAttrMapping) + val transferAttrMapping = attrMapping.map { + case (a1, a2) => (a1, updatedAttrMap.getOrElse(a2, a2)) + } + val newOtherAttrMapping = { + val existingAttrMappingSet = transferAttrMapping.map(_._2).toSet + newValidAttrMapping.filterNot { case (_, a) => existingAttrMappingSet.contains(a) } + } + planAfterRule -> (transferAttrMapping ++ newOtherAttrMapping).toSeq } } rewrite(this)._1 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala index 91ce187f4d270..404c8895c4d11 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala @@ -20,9 +20,11 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.dsl.plans -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ListQuery, Literal, NamedExpression} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Project, Union} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ListQuery, Literal, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan, Project, Union} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} import org.apache.spark.sql.types.IntegerType @@ -31,7 +33,7 @@ class QueryPlanSuite extends SparkFunSuite { test("origin remains the same after mapExpressions (SPARK-23823)") { CurrentOrigin.setPosition(0, 0) val column = AttributeReference("column", IntegerType)(NamedExpression.newExprId) - val query = plans.DslLogicalPlan(plans.table("table")).select(column) + val query = DslLogicalPlan(table("table")).select(column) CurrentOrigin.reset() val mappedQuery = query mapExpressions { @@ -83,4 +85,20 @@ class QueryPlanSuite extends SparkFunSuite { assert(countRelationsInPlan == 2) assert(countRelationsInPlanAndSubqueries == 5) } + + test("SPARK-33035: consecutive attribute updates in parent plan nodes") { + val testRule = new Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithNewOutput { + case p @ Project(projList, _) => + // Assigns new `ExprId`s for output references + val newPlan = p.copy(projectList = projList.map { ne => Alias(ne, ne.name)() }) + val attrMapping = p.output.zip(newPlan.output) + newPlan -> attrMapping + } + } + + val t = LocalRelation('a.int, 'b.int) + val plan = t.select($"a", $"b").select($"a", $"b").select($"a", $"b").analyze + assert(testRule(plan).resolved) + } } From 279334797234f5f83abd6879874b389e110920c2 Mon Sep 17 00:00:00 2001 From: "fqaiser94@gmail.com" Date: Tue, 6 Oct 2020 08:53:30 +0000 Subject: [PATCH 44/59] [SPARK-32511][SQL] Add dropFields method to Column class ### What changes were proposed in this pull request? 1. Refactored `WithFields` Expression to make it more extensible (now `UpdateFields`). 2. Added a new `dropFields` method to the `Column` class. This method should allow users to drop a `StructField` in a `StructType` column (with similar semantics to the `drop` method on `Dataset`). ### Why are the changes needed? Often Spark users have to work with deeply nested data e.g. to fix a data quality issue with an existing `StructField`. To do this with the existing Spark APIs, users have to rebuild the entire struct column. For example, let's say you have the following deeply nested data structure which has a data quality issue (`5` is missing): ``` import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val data = spark.createDataFrame(sc.parallelize( Seq(Row(Row(Row(1, 2, 3), Row(Row(4, null, 6), Row(7, 8, 9), Row(10, 11, 12)), Row(13, 14, 15))))), StructType(Seq( StructField("a", StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("b", StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("b", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))), StructField("c", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))) ))), StructField("c", StructType(Seq( StructField("a", IntegerType), StructField("b", IntegerType), StructField("c", IntegerType)))) )))))).cache data.show(false) +---------------------------------+ |a | +---------------------------------+ |[[1, 2, 3], [[4,, 6], [7, 8, 9]]]| +---------------------------------+ ``` Currently, to drop the missing value users would have to do something like this: ``` val result = data.withColumn("a", struct( $"a.a", struct( struct( $"a.b.a.a", $"a.b.a.c" ).as("a"), $"a.b.b", $"a.b.c" ).as("b"), $"a.c" )) result.show(false) +---------------------------------------------------------------+ |a | +---------------------------------------------------------------+ |[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]| +---------------------------------------------------------------+ ``` As you can see above, with the existing methods users must call the `struct` function and list all fields, including fields they don't want to change. This is not ideal as: >this leads to complex, fragile code that cannot survive schema evolution. [SPARK-16483](https://issues.apache.org/jira/browse/SPARK-16483) In contrast, with the method added in this PR, a user could simply do something like this to get the same result: ``` val result = data.withColumn("a", 'a.dropFields("b.a.b")) result.show(false) +---------------------------------------------------------------+ |a | +---------------------------------------------------------------+ |[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]| +---------------------------------------------------------------+ ``` This is the second of maybe 3 methods that could be added to the `Column` class to make it easier to manipulate nested data. Other methods under discussion in [SPARK-22231](https://issues.apache.org/jira/browse/SPARK-22231) include `withFieldRenamed`. However, this should be added in a separate PR. ### Does this PR introduce _any_ user-facing change? The documentation for `Column.withField` method has changed to include an additional note about how to write optimized queries when adding multiple nested Column directly. ### How was this patch tested? New unit tests were added. Jenkins must pass them. ### Related JIRAs: More discussion on this topic can be found here: - https://issues.apache.org/jira/browse/SPARK-22231 - https://issues.apache.org/jira/browse/SPARK-16483 Closes #29795 from fqaiser94/SPARK-32511-dropFields-second-try. Authored-by: fqaiser94@gmail.com Signed-off-by: Wenchen Fan --- .../expressions/complexTypeCreator.scala | 127 ++- .../sql/catalyst/optimizer/ComplexTypes.scala | 20 +- .../sql/catalyst/optimizer/Optimizer.scala | 6 +- .../{WithFields.scala => UpdateFields.scala} | 16 +- ...e.scala => CombineUpdateFieldsSuite.scala} | 41 +- .../optimizer/complexTypesSuite.scala | 345 ++++++- .../UpdateFieldsBenchmark-results.txt | 26 + .../scala/org/apache/spark/sql/Column.scala | 119 ++- .../spark/sql/ColumnExpressionSuite.scala | 881 ++++++++++++++++-- .../spark/sql/UpdateFieldsBenchmark.scala | 224 +++++ 10 files changed, 1607 insertions(+), 198 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{WithFields.scala => UpdateFields.scala} (68%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/{CombineWithFieldsSuite.scala => CombineUpdateFieldsSuite.scala} (65%) create mode 100644 sql/core/benchmarks/UpdateFieldsBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index c1471455b58c0..d5b1950e82c56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} +import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.{FUNC_ALIAS, FunctionBuilder} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -548,57 +550,114 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E } /** - * Adds/replaces field in struct by name. + * Represents an operation to be applied to the fields of a struct. */ -case class WithFields( - structExpr: Expression, - names: Seq[String], - valExprs: Seq[Expression]) extends Unevaluable { +trait StructFieldsOperation { + + val resolver: Resolver = SQLConf.get.resolver - assert(names.length == valExprs.length) + /** + * Returns an updated list of StructFields and Expressions that will ultimately be used + * as the fields argument for [[StructType]] and as the children argument for + * [[CreateNamedStruct]] respectively inside of [[UpdateFields]]. + */ + def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] +} + +/** + * Add or replace a field by name. + * + * We extend [[Unevaluable]] here to ensure that [[UpdateFields]] can include it as part of its + * children, and thereby enable the analyzer to resolve and transform valExpr as necessary. + */ +case class WithField(name: String, valExpr: Expression) + extends Unevaluable with StructFieldsOperation { + + override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = { + val newFieldExpr = (StructField(name, valExpr.dataType, valExpr.nullable), valExpr) + val result = ArrayBuffer.empty[(StructField, Expression)] + var hasMatch = false + for (existingFieldExpr @ (existingField, _) <- values) { + if (resolver(existingField.name, name)) { + hasMatch = true + result += newFieldExpr + } else { + result += existingFieldExpr + } + } + if (!hasMatch) result += newFieldExpr + result + } + + override def children: Seq[Expression] = valExpr :: Nil + + override def dataType: DataType = throw new IllegalStateException( + "WithField.dataType should not be called.") + + override def nullable: Boolean = throw new IllegalStateException( + "WithField.nullable should not be called.") + + override def prettyName: String = "WithField" +} + +/** + * Drop a field by name. + */ +case class DropField(name: String) extends StructFieldsOperation { + override def apply(values: Seq[(StructField, Expression)]): Seq[(StructField, Expression)] = + values.filterNot { case (field, _) => resolver(field.name, name) } +} + +/** + * Updates fields in a struct. + */ +case class UpdateFields(structExpr: Expression, fieldOps: Seq[StructFieldsOperation]) + extends Unevaluable { override def checkInputDataTypes(): TypeCheckResult = { - if (!structExpr.dataType.isInstanceOf[StructType]) { - TypeCheckResult.TypeCheckFailure( - "struct argument should be struct type, got: " + structExpr.dataType.catalogString) + val dataType = structExpr.dataType + if (!dataType.isInstanceOf[StructType]) { + TypeCheckResult.TypeCheckFailure("struct argument should be struct type, got: " + + dataType.catalogString) + } else if (newExprs.isEmpty) { + TypeCheckResult.TypeCheckFailure("cannot drop all fields in struct") } else { TypeCheckResult.TypeCheckSuccess } } - override def children: Seq[Expression] = structExpr +: valExprs + override def children: Seq[Expression] = structExpr +: fieldOps.collect { + case e: Expression => e + } - override def dataType: StructType = evalExpr.dataType.asInstanceOf[StructType] + override def dataType: StructType = StructType(newFields) override def nullable: Boolean = structExpr.nullable - override def prettyName: String = "with_fields" + override def prettyName: String = "update_fields" - lazy val evalExpr: Expression = { - val existingExprs = structExpr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map { - case (name, i) => (name, GetStructField(KnownNotNull(structExpr), i).asInstanceOf[Expression]) - } + private lazy val newFieldExprs: Seq[(StructField, Expression)] = { + val existingFieldExprs: Seq[(StructField, Expression)] = + structExpr.dataType.asInstanceOf[StructType].fields.zipWithIndex.map { + case (field, i) => (field, GetStructField(structExpr, i)) + } - val addOrReplaceExprs = names.zip(valExprs) - - val resolver = SQLConf.get.resolver - val newExprs = addOrReplaceExprs.foldLeft(existingExprs) { - case (resultExprs, newExpr @ (newExprName, _)) => - if (resultExprs.exists(x => resolver(x._1, newExprName))) { - resultExprs.map { - case (name, _) if resolver(name, newExprName) => newExpr - case x => x - } - } else { - resultExprs :+ newExpr - } - }.flatMap { case (name, expr) => Seq(Literal(name), expr) } + fieldOps.foldLeft(existingFieldExprs)((exprs, op) => op(exprs)) + } + + private lazy val newFields: Seq[StructField] = newFieldExprs.map(_._1) + + lazy val newExprs: Seq[Expression] = newFieldExprs.map(_._2) + + lazy val evalExpr: Expression = { + val createNamedStructExpr = CreateNamedStruct(newFieldExprs.flatMap { + case (field, expr) => Seq(Literal(field.name), expr) + }) - val expr = CreateNamedStruct(newExprs) if (structExpr.nullable) { - If(IsNull(structExpr), Literal(null, expr.dataType), expr) + If(IsNull(structExpr), Literal(null, dataType), createNamedStructExpr) } else { - expr + createNamedStructExpr } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala index 2aba4bae397c7..860219e55b052 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types.StructType /** * Simplify redundant [[CreateNamedStruct]], [[CreateArray]] and [[CreateMap]] expressions. @@ -39,18 +40,13 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] { // Remove redundant field extraction. case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => createNamedStruct.valExprs(ordinal) - case GetStructField(w @ WithFields(struct, names, valExprs), ordinal, maybeName) => - val name = w.dataType(ordinal).name - val matches = names.zip(valExprs).filter(_._1 == name) - if (matches.nonEmpty) { - // return last matching element as that is the final value for the field being extracted. - // For example, if a user submits a query like this: - // `$"struct_col".withField("b", lit(1)).withField("b", lit(2)).getField("b")` - // we want to return `lit(2)` (and not `lit(1)`). - val expr = matches.last._2 - If(IsNull(struct), Literal(null, expr.dataType), expr) - } else { - GetStructField(struct, ordinal, maybeName) + case GetStructField(u: UpdateFields, ordinal, _)if !u.structExpr.isInstanceOf[UpdateFields] => + val structExpr = u.structExpr + u.newExprs(ordinal) match { + // if the struct itself is null, then any value extracted from it (expr) will be null + // so we don't need to wrap expr in If(IsNull(struct), Literal(null, expr.dataType), expr) + case expr: GetStructField if expr.child.semanticEquals(structExpr) => expr + case expr => If(IsNull(structExpr), Literal(null, expr.dataType), expr) } // Remove redundant array indexing. case GetArrayStructFields(CreateArray(elems, useStringTypeWhenEmpty), field, ordinal, _, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f2360150e47b5..5bdaa504a3beb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -110,7 +110,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveRedundantAliases, UnwrapCastInBinaryComparison, RemoveNoopOperators, - CombineWithFields, + CombineUpdateFields, SimplifyExtractValueOps, OptimizeJsonExprs, CombineConcats) ++ @@ -223,7 +223,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+ - Batch("ReplaceWithFieldsExpression", Once, ReplaceWithFieldsExpression) + Batch("ReplaceUpdateFieldsExpression", Once, ReplaceUpdateFieldsExpression) // remove any batches with no rules. this may happen when subclasses do not add optional rules. batches.filter(_.rules.nonEmpty) @@ -257,7 +257,7 @@ abstract class Optimizer(catalogManager: CatalogManager) RewriteCorrelatedScalarSubquery.ruleName :: RewritePredicateSubquery.ruleName :: NormalizeFloatingNumbers.ruleName :: - ReplaceWithFieldsExpression.ruleName :: Nil + ReplaceUpdateFieldsExpression.ruleName :: Nil /** * Optimize all the subqueries inside expression. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala similarity index 68% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala index 05c90864e4bb0..c7154210e0c62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/WithFields.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala @@ -17,26 +17,26 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.WithFields +import org.apache.spark.sql.catalyst.expressions.UpdateFields import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule /** - * Combines all adjacent [[WithFields]] expression into a single [[WithFields]] expression. + * Combines all adjacent [[UpdateFields]] expression into a single [[UpdateFields]] expression. */ -object CombineWithFields extends Rule[LogicalPlan] { +object CombineUpdateFields extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case WithFields(WithFields(struct, names1, valExprs1), names2, valExprs2) => - WithFields(struct, names1 ++ names2, valExprs1 ++ valExprs2) + case UpdateFields(UpdateFields(struct, fieldOps1), fieldOps2) => + UpdateFields(struct, fieldOps1 ++ fieldOps2) } } /** - * Replaces [[WithFields]] expression with an evaluable expression. + * Replaces [[UpdateFields]] expression with an evaluable expression. */ -object ReplaceWithFieldsExpression extends Rule[LogicalPlan] { +object ReplaceUpdateFieldsExpression extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case w: WithFields => w.evalExpr + case u: UpdateFields => u.evalExpr } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineUpdateFieldsSuite.scala similarity index 65% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineUpdateFieldsSuite.scala index a3e0bbc57e639..ff9c60a2fa5bd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineWithFieldsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombineUpdateFieldsSuite.scala @@ -19,56 +19,53 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, WithFields} +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, UpdateFields, WithField} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -class CombineWithFieldsSuite extends PlanTest { +class CombineUpdateFieldsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - val batches = Batch("CombineWithFields", FixedPoint(10), CombineWithFields) :: Nil + val batches = Batch("CombineUpdateFields", FixedPoint(10), CombineUpdateFields) :: Nil } private val testRelation = LocalRelation('a.struct('a1.int)) - test("combines two WithFields") { + test("combines two adjacent UpdateFields Expressions") { val originalQuery = testRelation .select(Alias( - WithFields( - WithFields( + UpdateFields( + UpdateFields( 'a, - Seq("b1"), - Seq(Literal(4))), - Seq("c1"), - Seq(Literal(5))), "out")()) + WithField("b1", Literal(4)) :: Nil), + WithField("c1", Literal(5)) :: Nil), "out")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Alias(WithFields('a, Seq("b1", "c1"), Seq(Literal(4), Literal(5))), "out")()) + .select(Alias(UpdateFields('a, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: + Nil), "out")()) .analyze comparePlans(optimized, correctAnswer) } - test("combines three WithFields") { + test("combines three adjacent UpdateFields Expressions") { val originalQuery = testRelation .select(Alias( - WithFields( - WithFields( - WithFields( + UpdateFields( + UpdateFields( + UpdateFields( 'a, - Seq("b1"), - Seq(Literal(4))), - Seq("c1"), - Seq(Literal(5))), - Seq("d1"), - Seq(Literal(6))), "out")()) + WithField("b1", Literal(4)) :: Nil), + WithField("c1", Literal(5)) :: Nil), + WithField("d1", Literal(6)) :: Nil), "out")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Alias(WithFields('a, Seq("b1", "c1", "d1"), Seq(4, 5, 6).map(Literal(_))), "out")()) + .select(Alias(UpdateFields('a, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: + WithField("d1", Literal(6)) :: Nil), "out")()) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala index 00aed6a10cd64..d9cefdaf3fe70 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, OneRowRelation, Project, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.types._ @@ -37,14 +37,15 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { object Optimizer extends RuleExecutor[LogicalPlan] { val batches = Batch("collapse projections", FixedPoint(10), - CollapseProject) :: + CollapseProject) :: Batch("Constant Folding", FixedPoint(10), - NullPropagation, - ConstantFolding, - BooleanSimplification, - SimplifyConditionals, - SimplifyBinaryComparison, - SimplifyExtractValueOps) :: Nil + NullPropagation, + ConstantFolding, + BooleanSimplification, + SimplifyConditionals, + SimplifyBinaryComparison, + CombineUpdateFields, + SimplifyExtractValueOps) :: Nil } private val idAtt = ('id).long.notNull @@ -453,58 +454,182 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](3, 4))), null) } - private val structAttr = 'struct1.struct('a.int).withNullability(false) + private val structAttr = 'struct1.struct('a.int, 'b.int).withNullability(false) private val testStructRelation = LocalRelation(structAttr) - private val nullableStructAttr = 'struct1.struct('a.int) + private val nullableStructAttr = 'struct1.struct('a.int, 'b.int) private val testNullableStructRelation = LocalRelation(nullableStructAttr) - test("simplify GetStructField on WithFields that is not changing the attribute being extracted") { - def query(relation: LocalRelation): LogicalPlan = relation.select( - GetStructField(WithFields('struct1, Seq("b"), Seq(Literal(1))), 0, Some("a")) as "outerAttr") + test("simplify GetStructField on basic UpdateFields") { + def check(fieldOps: Seq[StructFieldsOperation], ordinal: Int, expected: Expression): Unit = { + def query(relation: LocalRelation): LogicalPlan = + relation.select(GetStructField(UpdateFields('struct1, fieldOps), ordinal).as("res")) + + checkRule( + query(testStructRelation), + testStructRelation.select(expected.as("res"))) + + checkRule( + query(testNullableStructRelation), + testNullableStructRelation.select((expected match { + case expr: GetStructField => expr + case expr => If(IsNull('struct1), Literal(null, expr.dataType), expr) + }).as("res"))) + } + + // scalastyle:off line.size.limit + + // add attribute, extract an attribute from the original struct + check(WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("c", Literal(3)) :: Nil, 1, GetStructField('struct1, 1)) + // add attribute, extract added attribute + check(WithField("c", Literal(3)) :: Nil, 2, Literal(3)) + + // replace attribute, extract an attribute from the original struct + check(WithField("a", Literal(1)) :: Nil, 1, GetStructField('struct1, 1)) + check(WithField("b", Literal(2)) :: Nil, 0, GetStructField('struct1, 0)) + // replace attribute, extract replaced attribute + check(WithField("a", Literal(1)) :: Nil, 0, Literal(1)) + check(WithField("b", Literal(2)) :: Nil, 1, Literal(2)) + + // add multiple attributes, extract an attribute from the original struct + check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 1, GetStructField('struct1, 1)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 1, GetStructField('struct1, 1)) + // add multiple attributes, extract newly added attribute + check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 2, Literal(4)) + check(WithField("c", Literal(4)) :: WithField("c", Literal(3)) :: Nil, 2, Literal(3)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 2, Literal(3)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 3, Literal(4)) + check(WithField("d", Literal(4)) :: WithField("c", Literal(3)) :: Nil, 2, Literal(4)) + check(WithField("d", Literal(4)) :: WithField("c", Literal(3)) :: Nil, 3, Literal(3)) + + // drop attribute, extract an attribute from the original struct + check(DropField("b") :: Nil, 0, GetStructField('struct1, 0)) + check(DropField("a") :: Nil, 0, GetStructField('struct1, 1)) + + // drop attribute, add attribute, extract an attribute from the original struct + check(DropField("b") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) + check(DropField("a") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 1)) + // drop attribute, add attribute, extract added attribute + check(DropField("b") :: WithField("c", Literal(3)) :: Nil, 1, Literal(3)) + check(DropField("a") :: WithField("c", Literal(3)) :: Nil, 1, Literal(3)) + + // add attribute, drop attribute, extract an attribute from the original struct + check(WithField("c", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField('struct1, 1)) + check(WithField("c", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField('struct1, 0)) + // add attribute, drop attribute, extract added attribute + check(WithField("c", Literal(3)) :: DropField("a") :: Nil, 1, Literal(3)) + check(WithField("c", Literal(3)) :: DropField("b") :: Nil, 1, Literal(3)) + + // replace attribute, drop same attribute, extract an attribute from the original struct + check(WithField("b", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("a", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField('struct1, 1)) + + // add attribute, drop same attribute, extract an attribute from the original struct + check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 1, GetStructField('struct1, 1)) + + // replace attribute, drop another attribute, extract added attribute + check(WithField("b", Literal(3)) :: DropField("a") :: Nil, 0, Literal(3)) + check(WithField("a", Literal(3)) :: DropField("b") :: Nil, 0, Literal(3)) + + // drop attribute, add same attribute, extract attribute from the original struct + check(DropField("b") :: WithField("b", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) + check(DropField("a") :: WithField("a", Literal(3)) :: Nil, 0, GetStructField('struct1, 1)) + // drop attribute, add same attribute, extract added attribute + check(DropField("b") :: WithField("b", Literal(3)) :: Nil, 1, Literal(3)) + check(DropField("a") :: WithField("a", Literal(3)) :: Nil, 1, Literal(3)) + + // drop non-existent attribute, add same attribute, extract attribute from the original struct + check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) + check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 1, GetStructField('struct1, 1)) + // drop non-existent attribute, add same attribute, extract added attribute + check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 2, Literal(3)) + + // scalastyle:on line.size.limit + } + + test("simplify GetStructField that is extracting a field nested inside a struct") { + val struct2 = 'struct2.struct('b.int) + val testStructRelation = LocalRelation(structAttr, struct2) + val testNullableStructRelation = LocalRelation(nullableStructAttr, struct2) + + // if the field being extracted is from the same struct that UpdateFields is modifying, + // we can just return GetStructField in both the non-nullable and nullable struct scenario + + def addFieldFromSameStructAndThenExtractIt(relation: LocalRelation): LogicalPlan = + relation.select(GetStructField( + UpdateFields('struct1, WithField("b", GetStructField('struct1, 0)) :: Nil), 1).as("res")) checkRule( - query(testStructRelation), - testStructRelation.select(GetStructField('struct1, 0, Some("a")) as "outerAttr")) + addFieldFromSameStructAndThenExtractIt(testStructRelation), + testStructRelation.select(GetStructField('struct1, 0).as("res"))) checkRule( - query(testNullableStructRelation), - testNullableStructRelation.select(GetStructField('struct1, 0, Some("a")) as "outerAttr")) - } + addFieldFromSameStructAndThenExtractIt(testNullableStructRelation), + testNullableStructRelation.select(GetStructField('struct1, 0).as("res"))) - test("simplify GetStructField on WithFields that is changing the attribute being extracted") { - def query(relation: LocalRelation): LogicalPlan = relation.select( - GetStructField(WithFields('struct1, Seq("b"), Seq(Literal(1))), 1, Some("b")) as "res") + // if the field being extracted is from a different struct than the one UpdateFields is + // modifying, we must return GetStructField wrapped in If(IsNull(struct), null, GetStructField) + // in the nullable struct scenario + + def addFieldFromAnotherStructAndThenExtractIt(relation: LocalRelation): LogicalPlan = + relation.select(GetStructField( + UpdateFields('struct1, WithField("b", GetStructField('struct2, 0)) :: Nil), 1).as("res")) checkRule( - query(testStructRelation), - testStructRelation.select(Literal(1) as "res")) + addFieldFromAnotherStructAndThenExtractIt(testStructRelation), + testStructRelation.select(GetStructField('struct2, 0).as("res"))) checkRule( - query(testNullableStructRelation), + addFieldFromAnotherStructAndThenExtractIt(testNullableStructRelation), testNullableStructRelation.select( - If(IsNull('struct1), Literal(null, IntegerType), Literal(1)) as "res")) + If(IsNull('struct1), Literal(null, IntegerType), GetStructField('struct2, 0)).as("res"))) } - test( - "simplify GetStructField on WithFields that is changing the attribute being extracted twice") { - def query(relation: LocalRelation): LogicalPlan = relation.select( - GetStructField(WithFields('struct1, Seq("b", "b"), Seq(Literal(1), Literal(2))), 1, Some("b")) - as "outerAtt") + test("simplify GetStructField on nested UpdateFields") { + def query(relation: LocalRelation, ordinal: Int): LogicalPlan = { + val nestedUpdateFields = + UpdateFields( + UpdateFields( + UpdateFields( + UpdateFields( + 'struct1, + WithField("c", Literal(1)) :: Nil), + WithField("d", Literal(2)) :: Nil), + WithField("e", Literal(3)) :: Nil), + WithField("f", Literal(4)) :: Nil) + + relation.select(GetStructField(nestedUpdateFields, ordinal) as "res") + } + + // extract newly added field checkRule( - query(testStructRelation), - testStructRelation.select(Literal(2) as "outerAtt")) + query(testStructRelation, 5), + testStructRelation.select(Literal(4) as "res")) checkRule( - query(testNullableStructRelation), + query(testNullableStructRelation, 5), testNullableStructRelation.select( - If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "outerAtt")) + If(IsNull('struct1), Literal(null, IntegerType), Literal(4)) as "res")) + + // extract field from original struct + + checkRule( + query(testStructRelation, 0), + testStructRelation.select(GetStructField('struct1, 0) as "res")) + + checkRule( + query(testNullableStructRelation, 0), + testNullableStructRelation.select(GetStructField('struct1, 0) as "res")) } - test("collapse multiple GetStructField on the same WithFields") { + test("simplify multiple GetStructField on the same UpdateFields") { def query(relation: LocalRelation): LogicalPlan = relation - .select(WithFields('struct1, Seq("b"), Seq(Literal(2))) as "struct2") + .select(UpdateFields('struct1, WithField("b", Literal(2)) :: Nil) as "struct2") .select( GetStructField('struct2, 0, Some("a")) as "struct1A", GetStructField('struct2, 1, Some("b")) as "struct1B") @@ -512,21 +637,21 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { checkRule( query(testStructRelation), testStructRelation.select( - GetStructField('struct1, 0, Some("a")) as "struct1A", + GetStructField('struct1, 0) as "struct1A", Literal(2) as "struct1B")) checkRule( query(testNullableStructRelation), testNullableStructRelation.select( - GetStructField('struct1, 0, Some("a")) as "struct1A", + GetStructField('struct1, 0) as "struct1A", If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct1B")) } - test("collapse multiple GetStructField on different WithFields") { + test("simplify multiple GetStructField on different UpdateFields") { def query(relation: LocalRelation): LogicalPlan = relation .select( - WithFields('struct1, Seq("b"), Seq(Literal(2))) as "struct2", - WithFields('struct1, Seq("b"), Seq(Literal(3))) as "struct3") + UpdateFields('struct1, WithField("b", Literal(2)) :: Nil) as "struct2", + UpdateFields('struct1, WithField("b", Literal(3)) :: Nil) as "struct3") .select( GetStructField('struct2, 0, Some("a")) as "struct2A", GetStructField('struct2, 1, Some("b")) as "struct2B", @@ -537,18 +662,148 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { query(testStructRelation), testStructRelation .select( - GetStructField('struct1, 0, Some("a")) as "struct2A", + GetStructField('struct1, 0) as "struct2A", Literal(2) as "struct2B", - GetStructField('struct1, 0, Some("a")) as "struct3A", + GetStructField('struct1, 0) as "struct3A", Literal(3) as "struct3B")) checkRule( query(testNullableStructRelation), testNullableStructRelation .select( - GetStructField('struct1, 0, Some("a")) as "struct2A", + GetStructField('struct1, 0) as "struct2A", If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B", - GetStructField('struct1, 0, Some("a")) as "struct3A", + GetStructField('struct1, 0) as "struct3A", If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B")) } + + test("simplify add multiple nested fields to non-nullable struct") { + // this scenario is possible if users add multiple nested columns to a non-nullable struct + // using the Column.withField API in a non-performant way + val structLevel2 = LocalRelation( + 'a1.struct( + 'a2.struct('a3.int.notNull)).notNull) + + val query = { + val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2", + UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2))))))) + + structLevel2.select( + UpdateFields( + addB3toA1A2, + Seq(WithField("a2", UpdateFields( + GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1")) + } + + val expected = structLevel2.select( + UpdateFields('a1, Seq( + // scalastyle:off line.size.limit + WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: Nil)), + WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil)) + // scalastyle:on line.size.limit + )).as("a1")) + + checkRule(query, expected) + } + + test("simplify add multiple nested fields to nullable struct") { + // this scenario is possible if users add multiple nested columns to a nullable struct + // using the Column.withField API in a non-performant way + val structLevel2 = LocalRelation( + 'a1.struct( + 'a2.struct('a3.int.notNull))) + + val query = { + val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2", + UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2))))))) + + structLevel2.select( + UpdateFields( + addB3toA1A2, + Seq(WithField("a2", UpdateFields( + GetStructField(addB3toA1A2, 0), Seq(WithField("c3", Literal(3))))))).as("a1")) + } + + val expected = { + val repeatedExpr = UpdateFields(GetStructField('a1, 0), WithField("b3", Literal(2)) :: Nil) + val repeatedExprDataType = StructType(Seq( + StructField("a3", IntegerType, nullable = false), + StructField("b3", IntegerType, nullable = false))) + + structLevel2.select( + UpdateFields('a1, Seq( + WithField("a2", repeatedExpr), + WithField("a2", UpdateFields( + If(IsNull('a1), Literal(null, repeatedExprDataType), repeatedExpr), + WithField("c3", Literal(3)) :: Nil)) + )).as("a1")) + } + + checkRule(query, expected) + } + + test("simplify drop multiple nested fields in non-nullable struct") { + // this scenario is possible if users drop multiple nested columns in a non-nullable struct + // using the Column.dropFields API in a non-performant way + val structLevel2 = LocalRelation( + 'a1.struct( + 'a2.struct('a3.int.notNull, 'b3.int.notNull, 'c3.int.notNull).notNull + ).notNull) + + val query = { + val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields( + GetStructField('a1, 0), Seq(DropField("b3")))))) + + structLevel2.select( + UpdateFields( + dropA1A2B, + Seq(WithField("a2", UpdateFields( + GetStructField(dropA1A2B, 0), Seq(DropField("c3")))))).as("a1")) + } + + val expected = structLevel2.select( + UpdateFields('a1, Seq( + WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3")))), + WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3"), DropField("c3")))) + )).as("a1")) + + checkRule(query, expected) + } + + test("simplify drop multiple nested fields in nullable struct") { + // this scenario is possible if users drop multiple nested columns in a nullable struct + // using the Column.dropFields API in a non-performant way + val structLevel2 = LocalRelation( + 'a1.struct( + 'a2.struct('a3.int.notNull, 'b3.int.notNull, 'c3.int.notNull) + )) + + val query = { + val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields( + GetStructField('a1, 0), Seq(DropField("b3")))))) + + structLevel2.select( + UpdateFields( + dropA1A2B, + Seq(WithField("a2", UpdateFields( + GetStructField(dropA1A2B, 0), Seq(DropField("c3")))))).as("a1")) + } + + val expected = { + val repeatedExpr = UpdateFields(GetStructField('a1, 0), DropField("b3") :: Nil) + val repeatedExprDataType = StructType(Seq( + StructField("a3", IntegerType, nullable = false), + StructField("c3", IntegerType, nullable = false))) + + structLevel2.select( + UpdateFields('a1, Seq( + WithField("a2", repeatedExpr), + WithField("a2", UpdateFields( + If(IsNull('a1), Literal(null, repeatedExprDataType), repeatedExpr), + DropField("c3") :: Nil)) + )).as("a1")) + } + + checkRule(query, expected) + } } diff --git a/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt new file mode 100644 index 0000000000000..5feca0e100bb1 --- /dev/null +++ b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt @@ -0,0 +1,26 @@ +================================================================================================ +Add 2 columns and drop 2 columns at 3 different depths of nesting +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Add 2 columns and drop 2 columns at 3 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------------------------------- +To non-nullable StructTypes using performant method 10 11 2 0.0 Infinity 1.0X +To nullable StructTypes using performant method 9 10 1 0.0 Infinity 1.0X +To non-nullable StructTypes using non-performant method 2457 2464 10 0.0 Infinity 0.0X +To nullable StructTypes using non-performant method 42641 43804 1644 0.0 Infinity 0.0X + + +================================================================================================ +Add 50 columns and drop 50 columns at 100 different depths of nesting +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_212-b03 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Add 50 columns and drop 50 columns at 100 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +----------------------------------------------------------------------------------------------------------------------------------------------------- +To non-nullable StructTypes using performant method 4595 4927 470 0.0 Infinity 1.0X +To nullable StructTypes using performant method 5185 5516 468 0.0 Infinity 0.9X + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index da542c67d9c51..a46d6c0bb2282 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -901,6 +901,23 @@ class Column(val expr: Expression) extends Logging { * // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields * }}} * + * This method supports adding/replacing nested fields directly e.g. + * + * {{{ + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4))) + * // result: {"a":{"a":1,"b":2,"c":3,"d":4}} + * }}} + * + * However, if you are going to add/replace multiple nested fields, it is more optimal to extract + * out the nested struct before adding/replacing multiple fields e.g. + * + * {{{ + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".withField("a", $"struct_col.a".withField("c", lit(3)).withField("d", lit(4)))) + * // result: {"a":{"a":1,"b":2,"c":3,"d":4}} + * }}} + * * @group expr_ops * @since 3.1.0 */ @@ -908,32 +925,102 @@ class Column(val expr: Expression) extends Logging { def withField(fieldName: String, col: Column): Column = withExpr { require(fieldName != null, "fieldName cannot be null") require(col != null, "col cannot be null") + updateFieldsHelper(expr, nameParts(fieldName), name => WithField(name, col.expr)) + } - val nameParts = if (fieldName.isEmpty) { + // scalastyle:off line.size.limit + /** + * An expression that drops fields in `StructType` by name. + * This is a no-op if schema doesn't contain field name(s). + * + * {{{ + * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + * df.select($"struct_col".dropFields("b")) + * // result: {"a":1} + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + * df.select($"struct_col".dropFields("c")) + * // result: {"a":1,"b":2} + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col") + * df.select($"struct_col".dropFields("b", "c")) + * // result: {"a":1} + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + * df.select($"struct_col".dropFields("a", "b")) + * // result: org.apache.spark.sql.AnalysisException: cannot resolve 'update_fields(update_fields(`struct_col`))' due to data type mismatch: cannot drop all fields in struct + * + * val df = sql("SELECT CAST(NULL AS struct) struct_col") + * df.select($"struct_col".dropFields("b")) + * // result: null of type struct + * + * val df = sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + * df.select($"struct_col".dropFields("b")) + * // result: {"a":1} + * + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".dropFields("a.b")) + * // result: {"a":{"a":1}} + * + * val df = sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + * df.select($"struct_col".dropFields("a.c")) + * // result: org.apache.spark.sql.AnalysisException: Ambiguous reference to fields + * }}} + * + * This method supports dropping multiple nested fields directly e.g. + * + * {{{ + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".dropFields("a.b", "a.c")) + * // result: {"a":{"a":1}} + * }}} + * + * However, if you are going to drop multiple nested fields, it is more optimal to extract + * out the nested struct before dropping multiple fields from it e.g. + * + * {{{ + * val df = sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + * df.select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c"))) + * // result: {"a":{"a":1}} + * }}} + * + * @group expr_ops + * @since 3.1.0 + */ + // scalastyle:on line.size.limit + def dropFields(fieldNames: String*): Column = withExpr { + def dropField(structExpr: Expression, fieldName: String): UpdateFields = + updateFieldsHelper(structExpr, nameParts(fieldName), name => DropField(name)) + + fieldNames.tail.foldLeft(dropField(expr, fieldNames.head)) { + (resExpr, fieldName) => dropField(resExpr, fieldName) + } + } + + private def nameParts(fieldName: String): Seq[String] = { + require(fieldName != null, "fieldName cannot be null") + + if (fieldName.isEmpty) { fieldName :: Nil } else { CatalystSqlParser.parseMultipartIdentifier(fieldName) } - withFieldHelper(expr, nameParts, Nil, col.expr) } - private def withFieldHelper( - struct: Expression, + private def updateFieldsHelper( + structExpr: Expression, namePartsRemaining: Seq[String], - namePartsDone: Seq[String], - value: Expression) : WithFields = { - val name = namePartsRemaining.head + valueFunc: String => StructFieldsOperation): UpdateFields = { + + val fieldName = namePartsRemaining.head if (namePartsRemaining.length == 1) { - WithFields(struct, name :: Nil, value :: Nil) + UpdateFields(structExpr, valueFunc(fieldName) :: Nil) } else { - val newNamesRemaining = namePartsRemaining.tail - val newNamesDone = namePartsDone :+ name - val newValue = withFieldHelper( - struct = UnresolvedExtractValue(struct, Literal(name)), - namePartsRemaining = newNamesRemaining, - namePartsDone = newNamesDone, - value = value) - WithFields(struct, name :: Nil, newValue :: Nil) + val newValue = updateFieldsHelper( + structExpr = UnresolvedExtractValue(structExpr, Literal(fieldName)), + namePartsRemaining = namePartsRemaining.tail, + valueFunc = valueFunc) + UpdateFields(structExpr, WithField(fieldName, newValue) :: Nil) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 24419968c0472..b11f4c603dfd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.scalatest.matchers.should.Matchers._ +import org.apache.spark.sql.UpdateFieldsBenchmark._ import org.apache.spark.sql.catalyst.expressions.{InSet, Literal, NamedExpression} import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ @@ -922,11 +923,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { assert(inSet.sql === "('a' IN ('a', 'b'))") } - def checkAnswerAndSchema( + def checkAnswer( df: => DataFrame, expectedAnswer: Seq[Row], expectedSchema: StructType): Unit = { - checkAnswer(df, expectedAnswer) assert(df.schema == expectedSchema) } @@ -940,8 +940,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil), StructType(Seq(StructField("a", structType, nullable = false)))) - private lazy val nullStructLevel1: DataFrame = spark.createDataFrame( - sparkContext.parallelize(Row(null) :: Nil), + private lazy val nullableStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Row(Row(1, null, 3)) :: Nil), StructType(Seq(StructField("a", structType, nullable = true)))) private lazy val structLevel2: DataFrame = spark.createDataFrame( @@ -951,12 +951,12 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("a", structType, nullable = false))), nullable = false)))) - private lazy val nullStructLevel2: DataFrame = spark.createDataFrame( - sparkContext.parallelize(Row(Row(null)) :: Nil), + private lazy val nullableStructLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3))) :: Nil), StructType(Seq( StructField("a", StructType(Seq( StructField("a", structType, nullable = true))), - nullable = false)))) + nullable = true)))) private lazy val structLevel3: DataFrame = spark.createDataFrame( sparkContext.parallelize(Row(Row(Row(Row(1, null, 3)))) :: Nil), @@ -1018,7 +1018,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field with no name") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("", lit(4))), Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( @@ -1031,7 +1031,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4))), Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( @@ -1043,10 +1043,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } - test("withField should add field to null struct") { - checkAnswerAndSchema( - nullStructLevel1.withColumn("a", $"a".withField("d", lit(4))), - Row(null) :: Nil, + test("withField should add field to nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", $"a".withField("d", lit(4))), + Row(null) :: Row(Row(1, null, 3, 4)) :: Nil, StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType, nullable = false), @@ -1056,10 +1056,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = true)))) } - test("withField should add field to nested null struct") { - checkAnswerAndSchema( - nullStructLevel2.withColumn("a", $"a".withField("a.d", lit(4))), - Row(Row(null)) :: Nil, + test("withField should add field to nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".withField("a.d", lit(4))), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3, 4))) :: Nil, StructType( Seq(StructField("a", StructType(Seq( StructField("a", StructType(Seq( @@ -1068,11 +1068,11 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("c", IntegerType, nullable = false), StructField("d", IntegerType, nullable = false))), nullable = true))), - nullable = false)))) + nullable = true)))) } test("withField should add null field to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(null).cast(IntegerType))), Row(Row(1, null, 3, null)) :: Nil, StructType(Seq( @@ -1085,7 +1085,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add multiple fields to struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("e", lit(5))), Row(Row(1, null, 3, 4, 5)) :: Nil, StructType(Seq( @@ -1098,12 +1098,26 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } + test("withField should add multiple fields to nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("e", lit(5))), + Row(null) :: Row(Row(1, null, 3, 4, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = true)))) + } + test("withField should add field to nested struct") { Seq( structLevel2.withColumn("a", 'a.withField("a.d", lit(4))), structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("d", lit(4)))) ).foreach { df => - checkAnswerAndSchema( + checkAnswer( df, Row(Row(Row(1, null, 3, 4))) :: Nil, StructType( @@ -1118,8 +1132,50 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } + test("withField should add multiple fields to nested struct") { + Seq( + col("a").withField("a", $"a.a".withField("d", lit(4)).withField("e", lit(5))), + col("a").withField("a.d", lit(4)).withField("a.e", lit(5)) + ).foreach { column => + checkAnswer( + structLevel2.select(column.as("a")), + Row(Row(Row(1, null, 3, 4, 5))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should add multiple fields to nested nullable struct") { + Seq( + col("a").withField("a", $"a.a".withField("d", lit(4)).withField("e", lit(5))), + col("a").withField("a.d", lit(4)).withField("a.e", lit(5)) + ).foreach { column => + checkAnswer( + nullableStructLevel2.select(column.as("a")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, null, 3, 4, 5))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + } + test("withField should add field to deeply nested struct") { - checkAnswerAndSchema( + checkAnswer( structLevel3.withColumn("a", 'a.withField("a.a.d", lit(4))), Row(Row(Row(Row(1, null, 3, 4)))) :: Nil, StructType(Seq( @@ -1136,7 +1192,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace field in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 2, 3)) :: Nil, StructType(Seq( @@ -1147,10 +1203,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } - test("withField should replace field in null struct") { - checkAnswerAndSchema( - nullStructLevel1.withColumn("a", 'a.withField("b", lit("foo"))), - Row(null) :: Nil, + test("withField should replace field in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("b", lit("foo"))), + Row(null) :: Row(Row(1, "foo", 3)) :: Nil, StructType(Seq( StructField("a", StructType(Seq( StructField("a", IntegerType, nullable = false), @@ -1159,10 +1215,10 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = true)))) } - test("withField should replace field in nested null struct") { - checkAnswerAndSchema( - nullStructLevel2.withColumn("a", $"a".withField("a.b", lit("foo"))), - Row(Row(null)) :: Nil, + test("withField should replace field in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".withField("a.b", lit("foo"))), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, "foo", 3))) :: Nil, StructType( Seq(StructField("a", StructType(Seq( StructField("a", StructType(Seq( @@ -1170,11 +1226,11 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("b", StringType, nullable = false), StructField("c", IntegerType, nullable = false))), nullable = true))), - nullable = false)))) + nullable = true)))) } test("withField should replace field with null value in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("c", lit(null).cast(IntegerType))), Row(Row(1, null, null)) :: Nil, StructType(Seq( @@ -1186,7 +1242,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace multiple fields in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("a", lit(10)).withField("b", lit(20))), Row(Row(10, 20, 3)) :: Nil, StructType(Seq( @@ -1197,12 +1253,24 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false)))) } + test("withField should replace multiple fields in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", 'a.withField("a", lit(10)).withField("b", lit(20))), + Row(null) :: Row(Row(10, 20, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true)))) + } + test("withField should replace field in nested struct") { Seq( structLevel2.withColumn("a", $"a".withField("a.b", lit(2))), structLevel2.withColumn("a", 'a.withField("a", $"a.a".withField("b", lit(2)))) ).foreach { df => - checkAnswerAndSchema( + checkAnswer( df, Row(Row(Row(1, 2, 3))) :: Nil, StructType(Seq( @@ -1216,8 +1284,46 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } } + test("withField should replace multiple fields in nested struct") { + Seq( + col("a").withField("a", $"a.a".withField("a", lit(10)).withField("b", lit(20))), + col("a").withField("a.a", lit(10)).withField("a.b", lit(20)) + ).foreach { column => + checkAnswer( + structLevel2.select(column.as("a")), + Row(Row(Row(10, 20, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("withField should replace multiple fields in nested nullable struct") { + Seq( + col("a").withField("a", $"a.a".withField("a", lit(10)).withField("b", lit(20))), + col("a").withField("a.a", lit(10)).withField("a.b", lit(20)) + ).foreach { column => + checkAnswer( + nullableStructLevel2.select(column.as("a")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(10, 20, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + } + test("withField should replace field in deeply nested struct") { - checkAnswerAndSchema( + checkAnswer( structLevel3.withColumn("a", $"a".withField("a.a.b", lit(2))), Row(Row(Row(Row(1, 2, 3)))) :: Nil, StructType(Seq( @@ -1242,7 +1348,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("b", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(100))), Row(Row(1, 100, 100)) :: Nil, StructType(Seq( @@ -1254,7 +1360,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should replace fields in struct in given order") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("b", lit(2)).withField("b", lit(20))), Row(Row(1, 20, 3)) :: Nil, StructType(Seq( @@ -1266,7 +1372,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("withField should add field and then replace same field in struct") { - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", 'a.withField("d", lit(4)).withField("d", lit(5))), Row(Row(1, null, 3, 5)) :: Nil, StructType(Seq( @@ -1290,7 +1396,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", 'a.withField("`a.b`.`e.f`", lit(2))), Row(Row(Row(1, 2, 3))) :: Nil, StructType(Seq( @@ -1317,7 +1423,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should replace field in struct even if casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), Row(Row(2, 1)) :: Nil, StructType(Seq( @@ -1326,7 +1432,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("B", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 2)) :: Nil, StructType(Seq( @@ -1339,7 +1445,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should add field to struct because casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("A", lit(2))), Row(Row(1, 1, 2)) :: Nil, StructType(Seq( @@ -1349,7 +1455,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructField("A", IntegerType, nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel1.withColumn("a", 'a.withField("b", lit(2))), Row(Row(1, 1, 2)) :: Nil, StructType(Seq( @@ -1377,7 +1483,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("withField should replace nested field in struct even if casing is different") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel2.withColumn("a", 'a.withField("A.a", lit(2))), Row(Row(Row(2, 1), Row(1, 1))) :: Nil, StructType(Seq( @@ -1392,7 +1498,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { nullable = false))), nullable = false)))) - checkAnswerAndSchema( + checkAnswer( mixedCaseStructLevel2.withColumn("a", 'a.withField("b.a", lit(2))), Row(Row(Row(1, 1), Row(2, 1))) :: Nil, StructType(Seq( @@ -1451,30 +1557,41 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") .select($"struct_col".withField("a.c", lit(3))) }.getMessage should include("Ambiguous reference to fields") + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a.c", lit(3)).withField("a.d", lit(4))), + Row(Row(Row(1, 2, 3, 4)))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".withField("a", + $"struct_col.a".withField("c", lit(3)).withField("d", lit(4)))), + Row(Row(Row(1, 2, 3, 4)))) } test("SPARK-32641: extracting field from non-null struct column after withField should return " + "field value") { // extract newly added field - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(4) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(4) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(3):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( structLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(3):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = false)))) @@ -1482,26 +1599,30 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("SPARK-32641: extracting field from null struct column after withField should return " + "null if the original struct was null") { + val nullStructLevel1 = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + // extract newly added field - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(null) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( nullStructLevel1.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) @@ -1514,27 +1635,671 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructType(Seq(StructField("a", structType, nullable = true)))) // extract newly added field - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("d", lit(4)).getField("d")), Row(4) :: Row(null) :: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // extract newly replaced field - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("a", lit(4)).getField("a")), Row(4) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // add new field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("d", lit(4)).getField("c")), Row(3) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) // replace field, extract another field from original struct - checkAnswerAndSchema( + checkAnswer( df.withColumn("a", $"a".withField("a", lit(4)).getField("c")), Row(3) :: Row(null):: Nil, StructType(Seq(StructField("a", IntegerType, nullable = true)))) } + + + test("dropFields should throw an exception if called on a non-StructType column") { + intercept[AnalysisException] { + testData.withColumn("key", $"key".dropFields("a")) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("dropFields should throw an exception if fieldName argument is null") { + intercept[IllegalArgumentException] { + structLevel1.withColumn("a", $"a".dropFields(null)) + }.getMessage should include("fieldName cannot be null") + } + + test("dropFields should throw an exception if any intermediate structs don't exist") { + intercept[AnalysisException] { + structLevel2.withColumn("a", 'a.dropFields("x.b")) + }.getMessage should include("No such struct field x in a") + + intercept[AnalysisException] { + structLevel3.withColumn("a", 'a.dropFields("a.x.b")) + }.getMessage should include("No such struct field x in a") + } + + test("dropFields should throw an exception if intermediate field is not a struct") { + intercept[AnalysisException] { + structLevel1.withColumn("a", 'a.dropFields("b.a")) + }.getMessage should include("struct argument should be struct type, got: int") + } + + test("dropFields should throw an exception if intermediate field reference is ambiguous") { + intercept[AnalysisException] { + val structLevel2: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false), + StructField("a", structType, nullable = false))), + nullable = false)))) + + structLevel2.withColumn("a", 'a.dropFields("a.b")) + }.getMessage should include("Ambiguous reference to fields") + } + + test("dropFields should drop field in struct") { + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nullable struct") { + checkAnswer( + nullableStructLevel1.withColumn("a", $"a".dropFields("b")), + Row(null) :: Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true)))) + } + + test("dropFields should drop multiple fields in struct") { + Seq( + structLevel1.withColumn("a", $"a".dropFields("b", "c")), + structLevel1.withColumn("a", 'a.dropFields("b").dropFields("c")) + ).foreach { df => + checkAnswer( + df, + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should throw an exception if no fields will be left in struct") { + intercept[AnalysisException] { + structLevel1.withColumn("a", 'a.dropFields("a", "b", "c")) + }.getMessage should include("cannot drop all fields in struct") + } + + test("dropFields should drop field with no name in struct") { + val structType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("", IntegerType, nullable = false))) + + val structLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Nil), + StructType(Seq(StructField("a", structType, nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", $"a".dropFields("")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nested struct") { + checkAnswer( + structLevel2.withColumn("a", 'a.dropFields("a.b")), + Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop multiple fields in nested struct") { + checkAnswer( + structLevel2.withColumn("a", 'a.dropFields("a.b", "a.c")), + Row(Row(Row(1))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".dropFields("a.b")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + + test("dropFields should drop multiple fields in nested nullable struct") { + checkAnswer( + nullableStructLevel2.withColumn("a", $"a".dropFields("a.b", "a.c")), + Row(null) :: Row(Row(null)) :: Row(Row(Row(1))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = true))), + nullable = true)))) + } + + test("dropFields should drop field in deeply nested struct") { + checkAnswer( + structLevel3.withColumn("a", 'a.dropFields("a.a.b")), + Row(Row(Row(Row(1, 3)))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop all fields with given name in struct") { + val structLevel1 = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2, 3)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + + test("dropFields should drop field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should not drop field in struct because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("A")), + Row(Row(1, 1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel1.withColumn("a", 'a.dropFields("b")), + Row(Row(1, 1)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("B", IntegerType, nullable = false))), + nullable = false)))) + } + } + + test("dropFields should drop nested field in struct even if casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer( + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")), + Row(Row(Row(1), Row(1, 1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("A", StructType(Seq( + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("B", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswer( + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")), + Row(Row(Row(1, 1), Row(1))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false), + StructField("b", StructType(Seq( + StructField("b", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + } + + test("dropFields should throw an exception because casing is different") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("A.a")) + }.getMessage should include("No such struct field A in a, B") + + intercept[AnalysisException] { + mixedCaseStructLevel2.withColumn("a", 'a.dropFields("b.a")) + }.getMessage should include("No such struct field b in a, B") + } + } + + test("dropFields should drop only fields that exist") { + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("d")), + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel1.withColumn("a", 'a.dropFields("b", "d")), + Row(Row(1, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + structLevel2.withColumn("a", $"a".dropFields("a.b", "a.d")), + Row(Row(Row(1, 3))) :: Nil, + StructType( + Seq(StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + } + + test("dropFields should drop multiple fields at arbitrary levels of nesting in a single call") { + val df: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil), + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", structType, nullable = false), + StructField("b", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + df.withColumn("a", $"a".dropFields("a.b", "b")), + Row(Row(Row(1, 3))) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("c", IntegerType, nullable = false))), nullable = false))), + nullable = false)))) + } + + test("dropFields user-facing examples") { + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("b")), + Row(Row(1))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("c")), + Row(Row(1, 2))) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'c', 3) struct_col") + .select($"struct_col".dropFields("b", "c")), + Row(Row(1))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', 1, 'b', 2) struct_col") + .select($"struct_col".dropFields("a", "b")) + }.getMessage should include("cannot drop all fields in struct") + + checkAnswer( + sql("SELECT CAST(NULL AS struct) struct_col") + .select($"struct_col".dropFields("b")), + Row(null)) + + checkAnswer( + sql("SELECT named_struct('a', 1, 'b', 2, 'b', 3) struct_col") + .select($"struct_col".dropFields("b")), + Row(Row(1))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2)) struct_col") + .select($"struct_col".dropFields("a.b")), + Row(Row(Row(1)))) + + intercept[AnalysisException] { + sql("SELECT named_struct('a', named_struct('b', 1), 'a', named_struct('c', 2)) struct_col") + .select($"struct_col".dropFields("a.c")) + }.getMessage should include("Ambiguous reference to fields") + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col") + .select($"struct_col".dropFields("a.b", "a.c")), + Row(Row(Row(1)))) + + checkAnswer( + sql("SELECT named_struct('a', named_struct('a', 1, 'b', 2, 'c', 3)) struct_col") + .select($"struct_col".withField("a", $"struct_col.a".dropFields("b", "c"))), + Row(Row(Row(1)))) + } + + test("should correctly handle different dropField + withField + getField combinations") { + val structType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = false))) + + val structLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Nil), + StructType(Seq(StructField("a", structType, nullable = false)))) + + val nullStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + + val nullableStructLevel1: DataFrame = spark.createDataFrame( + sparkContext.parallelize(Row(Row(1, 2)) :: Row(null) :: Nil), + StructType(Seq(StructField("a", structType, nullable = true)))) + + def check( + fieldOps: Column => Column, + getFieldName: String, + expectedValue: Option[Int]): Unit = { + + def query(df: DataFrame): DataFrame = + df.select(fieldOps(col("a")).getField(getFieldName).as("res")) + + checkAnswer( + query(structLevel1), + Row(expectedValue.orNull) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = expectedValue.isEmpty)))) + + checkAnswer( + query(nullStructLevel1), + Row(null) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + + checkAnswer( + query(nullableStructLevel1), + Row(expectedValue.orNull) :: Row(null) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + } + + // add attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)), "a", Some(1)) + check(_.withField("c", lit(3)), "b", Some(2)) + + // add attribute, extract added attribute + check(_.withField("c", lit(3)), "c", Some(3)) + check(_.withField("c", col("a.a")), "c", Some(1)) + check(_.withField("c", col("a.b")), "c", Some(2)) + check(_.withField("c", lit(null).cast(IntegerType)), "c", None) + + // replace attribute, extract an attribute from the original struct + check(_.withField("b", lit(3)), "a", Some(1)) + check(_.withField("a", lit(3)), "b", Some(2)) + + // replace attribute, extract replaced attribute + check(_.withField("b", lit(3)), "b", Some(3)) + check(_.withField("b", lit(null).cast(IntegerType)), "b", None) + check(_.withField("a", lit(3)), "a", Some(3)) + check(_.withField("a", lit(null).cast(IntegerType)), "a", None) + + // drop attribute, extract an attribute from the original struct + check(_.dropFields("b"), "a", Some(1)) + check(_.dropFields("a"), "b", Some(2)) + + // drop attribute, add attribute, extract an attribute from the original struct + check(_.dropFields("b").withField("c", lit(3)), "a", Some(1)) + check(_.dropFields("a").withField("c", lit(3)), "b", Some(2)) + + // drop attribute, add another attribute, extract added attribute + check(_.dropFields("a").withField("c", lit(3)), "c", Some(3)) + check(_.dropFields("b").withField("c", lit(3)), "c", Some(3)) + + // add attribute, drop attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)).dropFields("a"), "b", Some(2)) + check(_.withField("c", lit(3)).dropFields("b"), "a", Some(1)) + + // add attribute, drop another attribute, extract added attribute + check(_.withField("c", lit(3)).dropFields("a"), "c", Some(3)) + check(_.withField("c", lit(3)).dropFields("b"), "c", Some(3)) + + // replace attribute, drop same attribute, extract an attribute from the original struct + check(_.withField("b", lit(3)).dropFields("b"), "a", Some(1)) + check(_.withField("a", lit(3)).dropFields("a"), "b", Some(2)) + + // add attribute, drop same attribute, extract an attribute from the original struct + check(_.withField("c", lit(3)).dropFields("c"), "a", Some(1)) + check(_.withField("c", lit(3)).dropFields("c"), "b", Some(2)) + + // add attribute, drop another attribute, extract added attribute + check(_.withField("b", lit(3)).dropFields("a"), "b", Some(3)) + check(_.withField("a", lit(3)).dropFields("b"), "a", Some(3)) + check(_.withField("b", lit(null).cast(IntegerType)).dropFields("a"), "b", None) + check(_.withField("a", lit(null).cast(IntegerType)).dropFields("b"), "a", None) + + // drop attribute, add same attribute, extract added attribute + check(_.dropFields("b").withField("b", lit(3)), "b", Some(3)) + check(_.dropFields("a").withField("a", lit(3)), "a", Some(3)) + check(_.dropFields("b").withField("b", lit(null).cast(IntegerType)), "b", None) + check(_.dropFields("a").withField("a", lit(null).cast(IntegerType)), "a", None) + check(_.dropFields("c").withField("c", lit(3)), "c", Some(3)) + + // add attribute, drop same attribute, add same attribute again, extract added attribute + check(_.withField("c", lit(3)).dropFields("c").withField("c", lit(4)), "c", Some(4)) + } + + test("should move field up one level of nesting") { + // move a field up one level + checkAnswer( + nullableStructLevel2.select( + col("a").withField("c", col("a.a.c")).dropFields("a.c").as("res")), + Row(null) :: Row(Row(null, null)) :: Row(Row(Row(1, null), 3)) :: Nil, + StructType(Seq( + StructField("res", StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true))), + nullable = true), + StructField("c", IntegerType, nullable = true))), + nullable = true)))) + + // move a field up one level and then extract it + checkAnswer( + nullableStructLevel2.select( + col("a").withField("c", col("a.a.c")).dropFields("a.c").getField("c").as("res")), + Row(null) :: Row(null) :: Row(3) :: Nil, + StructType(Seq(StructField("res", IntegerType, nullable = true)))) + } + + test("should be able to refer to newly added nested column") { + intercept[AnalysisException] { + structLevel1.select($"a".withField("d", lit(4)).withField("e", $"a.d" + 1).as("a")) + }.getMessage should include("No such struct field d in a, b, c") + + checkAnswer( + structLevel1 + .select($"a".withField("d", lit(4)).as("a")) + .select($"a".withField("e", $"a.d" + 1).as("a")), + Row(Row(1, null, 3, 4, 5)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = false), + StructField("e", IntegerType, nullable = false))), + nullable = false)))) + } + + test("should be able to drop newly added nested column") { + Seq( + structLevel1.select($"a".withField("d", lit(4)).dropFields("d").as("a")), + structLevel1 + .select($"a".withField("d", lit(4)).as("a")) + .select($"a".dropFields("d").as("a")) + ).foreach { query => + checkAnswer( + query, + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", structType, nullable = false)))) + } + } + + test("should still be able to refer to dropped column within the same select statement") { + // we can still access the nested column even after dropping it within the same select statement + checkAnswer( + structLevel1.select($"a".dropFields("c").withField("z", $"a.c").as("a")), + Row(Row(1, null, 3)) :: Nil, + StructType(Seq( + StructField("a", StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", IntegerType, nullable = true), + StructField("z", IntegerType, nullable = false))), + nullable = false)))) + + // we can't access the nested column in subsequent select statement after dropping it in a + // previous select statement + intercept[AnalysisException]{ + structLevel1 + .select($"a".dropFields("c").as("a")) + .select($"a".withField("z", $"a.c")).as("a") + }.getMessage should include("No such struct field c in a, b;") + } + + test("nestedDf should generate nested DataFrames") { + checkAnswer( + emptyNestedDf(1, 1, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(1, 2, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", IntegerType, nullable = false), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 1, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false))), + nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 2, nullable = false), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false), + StructField("nested2Col1", IntegerType, nullable = false))), + nullable = false), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = false)))) + + checkAnswer( + emptyNestedDf(2, 2, nullable = true), + Seq.empty[Row], + StructType(Seq(StructField("nested0Col0", StructType(Seq( + StructField("nested1Col0", StructType(Seq( + StructField("nested2Col0", IntegerType, nullable = false), + StructField("nested2Col1", IntegerType, nullable = false))), + nullable = true), + StructField("nested1Col1", IntegerType, nullable = false))), + nullable = true)))) + } + + Seq(Performant, NonPerformant).foreach { method => + Seq(false, true).foreach { nullable => + test(s"should add and drop 1 column at each depth of nesting using ${method.name} method, " + + s"nullable = $nullable") { + val maxDepth = 3 + + // dataframe with nested*Col0 to nested*Col2 at each depth + val inputDf = emptyNestedDf(maxDepth, 3, nullable) + + // add nested*Col3 and drop nested*Col2 + val modifiedColumn = method( + column = col(nestedColName(0, 0)), + numsToAdd = Seq(3), + numsToDrop = Seq(2), + maxDepth = maxDepth + ).as(nestedColName(0, 0)) + val resultDf = inputDf.select(modifiedColumn) + + // dataframe with nested*Col0, nested*Col1, nested*Col3 at each depth + val expectedDf = { + val colNums = Seq(0, 1, 3) + val nestedColumnDataType = nestedStructType(colNums, nullable, maxDepth) + + spark.createDataFrame( + spark.sparkContext.emptyRDD[Row], + StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable)))) + } + + checkAnswer(resultDf, expectedDf.collect(), expectedDf.schema) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala new file mode 100644 index 0000000000000..28af552fe586b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala @@ -0,0 +1,224 @@ +/* + * 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.spark.sql + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +/** + * Benchmark to measure Spark's performance analyzing and optimizing long UpdateFields chains. + * + * {{{ + * To run this benchmark: + * 1. without sbt: + * bin/spark-submit --class + * 2. with sbt: + * build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/UpdateFieldsBenchmark-results.txt". + * }}} + */ +object UpdateFieldsBenchmark extends SqlBasedBenchmark { + + def nestedColName(d: Int, colNum: Int): String = s"nested${d}Col$colNum" + + def nestedStructType( + colNums: Seq[Int], + nullable: Boolean, + maxDepth: Int, + currDepth: Int = 1): StructType = { + + if (currDepth == maxDepth) { + val fields = colNums.map { colNum => + val name = nestedColName(currDepth, colNum) + StructField(name, IntegerType, nullable = false) + } + StructType(fields) + } else { + val fields = colNums.foldLeft(Seq.empty[StructField]) { + case (structFields, colNum) if colNum == 0 => + val nested = nestedStructType(colNums, nullable, maxDepth, currDepth + 1) + structFields :+ StructField(nestedColName(currDepth, colNum), nested, nullable) + case (structFields, colNum) => + val name = nestedColName(currDepth, colNum) + structFields :+ StructField(name, IntegerType, nullable = false) + } + StructType(fields) + } + } + + /** + * Utility function for generating an empty DataFrame with nested columns. + * + * @param maxDepth: The depth to which to create nested columns. + * @param numColsAtEachDepth: The number of columns to create at each depth. + * @param nullable: This value is used to set the nullability of any StructType columns. + */ + def emptyNestedDf(maxDepth: Int, numColsAtEachDepth: Int, nullable: Boolean): DataFrame = { + require(maxDepth > 0) + require(numColsAtEachDepth > 0) + + val nestedColumnDataType = nestedStructType(0 until numColsAtEachDepth, nullable, maxDepth) + spark.createDataFrame( + spark.sparkContext.emptyRDD[Row], + StructType(Seq(StructField(nestedColName(0, 0), nestedColumnDataType, nullable)))) + } + + trait ModifyNestedColumns { + val name: String + def apply(column: Column, numsToAdd: Seq[Int], numsToDrop: Seq[Int], maxDepth: Int): Column + } + + object Performant extends ModifyNestedColumns { + override val name: String = "performant" + + override def apply( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int): Column = helper(column, numsToAdd, numsToDrop, maxDepth, 1) + + private def helper( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int, + currDepth: Int): Column = { + + // drop columns at the current depth + val dropped = if (numsToDrop.nonEmpty) { + column.dropFields(numsToDrop.map(num => nestedColName(currDepth, num)): _*) + } else column + + // add columns at the current depth + val added = numsToAdd.foldLeft(dropped) { + (res, num) => res.withField(nestedColName(currDepth, num), lit(num)) + } + + if (currDepth == maxDepth) { + added + } else { + // add/drop columns at the next depth + val newValue = helper( + column = col((0 to currDepth).map(d => nestedColName(d, 0)).mkString(".")), + numsToAdd = numsToAdd, + numsToDrop = numsToDrop, + currDepth = currDepth + 1, + maxDepth = maxDepth) + added.withField(nestedColName(currDepth, 0), newValue) + } + } + } + + object NonPerformant extends ModifyNestedColumns { + override val name: String = "non-performant" + + override def apply( + column: Column, + numsToAdd: Seq[Int], + numsToDrop: Seq[Int], + maxDepth: Int): Column = { + + val dropped = if (numsToDrop.nonEmpty) { + val colsToDrop = (1 to maxDepth).flatMap { depth => + numsToDrop.map(num => s"${prefix(depth)}${nestedColName(depth, num)}") + } + column.dropFields(colsToDrop: _*) + } else column + + val added = { + val colsToAdd = (1 to maxDepth).flatMap { depth => + numsToAdd.map(num => (s"${prefix(depth)}${nestedColName(depth, num)}", lit(num))) + } + colsToAdd.foldLeft(dropped)((col, add) => col.withField(add._1, add._2)) + } + + added + } + + private def prefix(depth: Int): String = + if (depth == 1) "" + else (1 until depth).map(d => nestedColName(d, 0)).mkString("", ".", ".") + } + + private def updateFieldsBenchmark( + methods: Seq[ModifyNestedColumns], + maxDepth: Int, + initialNumberOfColumns: Int, + numsToAdd: Seq[Int] = Seq.empty, + numsToDrop: Seq[Int] = Seq.empty): Unit = { + + val name = s"Add ${numsToAdd.length} columns and drop ${numsToDrop.length} columns " + + s"at $maxDepth different depths of nesting" + + runBenchmark(name) { + val benchmark = new Benchmark( + name = name, + // The purpose of this benchmark is to ensure Spark is able to analyze and optimize long + // UpdateFields chains quickly so it runs over 0 rows of data. + valuesPerIteration = 0, + output = output) + + val nonNullableStructsDf = emptyNestedDf(maxDepth, initialNumberOfColumns, nullable = false) + val nullableStructsDf = emptyNestedDf(maxDepth, initialNumberOfColumns, nullable = true) + + methods.foreach { method => + val modifiedColumn = method( + column = col(nestedColName(0, 0)), + numsToAdd = numsToAdd, + numsToDrop = numsToDrop, + maxDepth = maxDepth + ).as(nestedColName(0, 0)) + + benchmark.addCase(s"To non-nullable StructTypes using ${method.name} method") { _ => + nonNullableStructsDf.select(modifiedColumn).queryExecution.optimizedPlan + } + + benchmark.addCase(s"To nullable StructTypes using ${method.name} method") { _ => + nullableStructsDf.select(modifiedColumn).queryExecution.optimizedPlan + } + } + + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + // This benchmark compares the performant and non-performant methods of writing the same query. + // We use small values for maxDepth, numsToAdd, and numsToDrop because the NonPerformant method + // scales extremely poorly with the number of nested columns being added/dropped. + updateFieldsBenchmark( + methods = Seq(Performant, NonPerformant), + maxDepth = 3, + initialNumberOfColumns = 5, + numsToAdd = 5 to 6, + numsToDrop = 3 to 4) + + // This benchmark is to show that the performant method of writing a query when we want to add + // and drop a large number of nested columns scales nicely. + updateFieldsBenchmark( + methods = Seq(Performant), + maxDepth = 100, + initialNumberOfColumns = 51, + numsToAdd = 51 to 100, + numsToDrop = 1 to 50) + } +} From ddc7012b3d4cd05c6695378989c9d1a78102bbbd Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 6 Oct 2020 09:09:19 +0000 Subject: [PATCH 45/59] [SPARK-32243][SQL] HiveSessionCatalog call super.makeFunctionExpression should throw earlier when got Spark UDAF Invalid arguments number error MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? When we create a UDAF function use class extended `UserDefinedAggregeteFunction`,  when we call the function,  in support hive mode, in HiveSessionCatalog, it will call super.makeFunctionExpression,  but it will catch error  such as the function need 2 parameter and we only give 1, throw exception only show  ``` No handler for UDF/UDAF/UDTF xxxxxxxx ``` This is confused for develop , we should show error thrown by super method too, For this pr's UT : Before change, throw Exception like ``` No handler for UDF/UDAF/UDTF 'org.apache.spark.sql.hive.execution.LongProductSum'; line 1 pos 7 ``` After this pr, throw exception ``` Spark UDAF Error: Invalid number of arguments for function longProductSum. Expected: 2; Found: 1; Hive UDF/UDAF/UDTF Error: No handler for UDF/UDAF/UDTF 'org.apache.spark.sql.hive.execution.LongProductSum'; line 1 pos 7 ``` ### Why are the changes needed? Show more detail error message when define UDAF ### Does this PR introduce _any_ user-facing change? People will see more detail error message when use spark sql's UDAF in hive support Mode ### How was this patch tested? Added UT Closes #29054 from AngersZhuuuu/SPARK-32243. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- .../catalog/InvalidUDFClassException.scala | 28 +++++ .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../spark/sql/hive/HiveSessionCatalog.scala | 103 ++++++++++-------- .../sql/hive/execution/HiveUDAFSuite.scala | 14 +++ 4 files changed, 102 insertions(+), 45 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala new file mode 100644 index 0000000000000..bc02efd5113c2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.sql.catalyst.catalog + +import org.apache.spark.sql.AnalysisException + +/** + * Thrown when a query failed for invalid function class, usually because a SQL + * function's class does not follow the rules of the UDF/UDAF/UDTF class definition. + */ +class InvalidUDFClassException private[sql](message: String) + extends AnalysisException(message, None, None, None, None) { +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e9a02c15f7362..4865629329831 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1335,7 +1335,7 @@ class SessionCatalog( } e } else { - throw new AnalysisException(s"No handler for UDAF '${clazz.getCanonicalName}'. " + + throw new InvalidUDFClassException(s"No handler for UDAF '${clazz.getCanonicalName}'. " + s"Use sparkSession.udf.register(...) instead.") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index bc7760c982aab..f24834b938a1e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, Gener import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry -import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, ExternalCatalog, FunctionResourceLoader, GlobalTempViewManager, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper @@ -57,6 +57,56 @@ private[sql] class HiveSessionCatalog( parser, functionResourceLoader) { + private def makeHiveFunctionExpression( + name: String, + clazz: Class[_], + input: Seq[Expression]): Expression = { + var udfExpr: Option[Expression] = None + try { + // When we instantiate hive UDF wrapper class, we may throw exception if the input + // expressions don't satisfy the hive UDF, such as type mismatch, input number + // mismatch, etc. Here we catch the exception and throw AnalysisException instead. + if (classOf[UDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[UDAF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction( + name, + new HiveFunctionWrapper(clazz.getName), + input, + isUDAFBridgeRequired = true)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) + // Force it to check data types. + udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema + } + } catch { + case NonFatal(e) => + val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" + val errorMsg = + if (classOf[GenericUDTF].isAssignableFrom(clazz)) { + s"$noHandlerMsg\nPlease make sure your function overrides " + + "`public StructObjectInspector initialize(ObjectInspector[] args)`." + } else { + noHandlerMsg + } + val analysisException = new AnalysisException(errorMsg) + analysisException.setStackTrace(e.getStackTrace) + throw analysisException + } + udfExpr.getOrElse { + throw new InvalidUDFClassException( + s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") + } + } + /** * Constructs a [[Expression]] based on the provided class that represents a function. * @@ -69,49 +119,14 @@ private[sql] class HiveSessionCatalog( // Current thread context classloader may not be the one loaded the class. Need to switch // context classloader to initialize instance properly. Utils.withContextClassLoader(clazz.getClassLoader) { - Try(super.makeFunctionExpression(name, clazz, input)).getOrElse { - var udfExpr: Option[Expression] = None - try { - // When we instantiate hive UDF wrapper class, we may throw exception if the input - // expressions don't satisfy the hive UDF, such as type mismatch, input number - // mismatch, etc. Here we catch the exception and throw AnalysisException instead. - if (classOf[UDF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { - udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[UDAF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveUDAFFunction( - name, - new HiveFunctionWrapper(clazz.getName), - input, - isUDAFBridgeRequired = true)) - udfExpr.get.dataType // Force it to check input data types. - } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { - udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) - udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema // Force it to check data types. - } - } catch { - case NonFatal(e) => - val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" - val errorMsg = - if (classOf[GenericUDTF].isAssignableFrom(clazz)) { - s"$noHandlerMsg\nPlease make sure your function overrides " + - "`public StructObjectInspector initialize(ObjectInspector[] args)`." - } else { - noHandlerMsg - } - val analysisException = new AnalysisException(errorMsg) - analysisException.setStackTrace(e.getStackTrace) - throw analysisException - } - udfExpr.getOrElse { - throw new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") - } + try { + super.makeFunctionExpression(name, clazz, input) + } catch { + // If `super.makeFunctionExpression` throw `InvalidUDFClassException`, we construct + // Hive UDF/UDAF/UDTF with function definition. Otherwise, we just throw it earlier. + case _: InvalidUDFClassException => + makeHiveFunctionExpression(name, clazz, input) + case e => throw e } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index 9e33a8ee4cc5c..ed44dcd8d7a29 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -161,6 +161,20 @@ class HiveUDAFSuite extends QueryTest checkAnswer(sql("select histogram_numeric(a,2) from abc where a=3"), Row(null)) } } + + test("SPARK-32243: Spark UDAF Invalid arguments number error should throw earlier") { + // func need two arguments + val functionName = "longProductSum" + val functionClass = "org.apache.spark.sql.hive.execution.LongProductSum" + withUserDefinedFunction(functionName -> true) { + sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'") + val e = intercept[AnalysisException] { + sql(s"SELECT $functionName(100)") + }.getMessage + assert(e.contains( + s"Invalid number of arguments for function $functionName. Expected: 2; Found: 1;")) + } + } } /** From 0812d6c17cc4876bb87a9d1fec35ec8c7b2365f0 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Tue, 6 Oct 2020 18:11:24 +0900 Subject: [PATCH 46/59] [SPARK-33073][PYTHON] Improve error handling on Pandas to Arrow conversion failures ### What changes were proposed in this pull request? This improves error handling when a failure in conversion from Pandas to Arrow occurs. And fixes tests to be compatible with upcoming Arrow 2.0.0 release. ### Why are the changes needed? Current tests will fail with Arrow 2.0.0 because of a change in error message when the schema is invalid. For these cases, the current error message also includes information on disabling safe conversion config, which is mainly meant for floating point truncation and overflow. The tests have been updated to use a message that is show for past Arrow versions, and upcoming. If the user enters an invalid schema, the error produced by pyarrow is not consistent and either `TypeError` or `ArrowInvalid`, with the latter being caught, and raised as a `RuntimeError` with the extra info. The error handling is improved by: - narrowing the exception type to `TypeError`s, which `ArrowInvalid` is a subclass and what is raised on safe conversion failures. - The exception is only raised with additional information on disabling "spark.sql.execution.pandas.convertToArrowArraySafely" if it is enabled in the first place. - The original exception is chained to better show it to the user. ### Does this PR introduce _any_ user-facing change? Yes, the error re-raised changes from a RuntimeError to a ValueError, which better categorizes this type of error and in-line with the original Arrow error. ### How was this patch tested? Existing tests, using pyarrow 1.0.1 and 2.0.0-snapshot Closes #29951 from BryanCutler/arrow-better-handle-pandas-errors-SPARK-33073. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- python/pyspark/sql/pandas/serializers.py | 17 ++++++++++------- python/pyspark/sql/tests/test_arrow.py | 9 +++++---- .../sql/tests/test_pandas_grouped_map.py | 15 ++++++++------- 3 files changed, 23 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 4b91c6a0f8730..63fb8562799e3 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -153,13 +153,16 @@ def create_array(s, t): s = s.astype(s.dtypes.categories.dtype) try: array = pa.Array.from_pandas(s, mask=mask, type=t, safe=self._safecheck) - except pa.ArrowException as e: - error_msg = "Exception thrown when converting pandas.Series (%s) to Arrow " + \ - "Array (%s). It can be caused by overflows or other unsafe " + \ - "conversions warned by Arrow. Arrow safe type check can be " + \ - "disabled by using SQL config " + \ - "`spark.sql.execution.pandas.convertToArrowArraySafely`." - raise RuntimeError(error_msg % (s.dtype, t), e) + except ValueError as e: + if self._safecheck: + error_msg = "Exception thrown when converting pandas.Series (%s) to " + \ + "Arrow Array (%s). It can be caused by overflows or other " + \ + "unsafe conversions warned by Arrow. Arrow safe type check " + \ + "can be disabled by using SQL config " + \ + "`spark.sql.execution.pandas.convertToArrowArraySafely`." + raise ValueError(error_msg % (s.dtype, t)) from e + else: + raise e return array arrs = [] diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index c6497923d84fb..55d5e9017b345 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -264,11 +264,12 @@ def test_createDataFrame_with_schema(self): def test_createDataFrame_with_incorrect_schema(self): pdf = self.create_pandas_data_frame() fields = list(self.schema) - fields[0], fields[1] = fields[1], fields[0] # swap str with int + fields[5], fields[6] = fields[6], fields[5] # swap decimal with date wrong_schema = StructType(fields) - with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, "integer.*required"): - self.spark.createDataFrame(pdf, schema=wrong_schema) + with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": False}): + with QuietTest(self.sc): + with self.assertRaisesRegexp(Exception, "[D|d]ecimal.*got.*date"): + self.spark.createDataFrame(pdf, schema=wrong_schema) def test_createDataFrame_with_names(self): pdf = self.create_pandas_data_frame() diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py index 81b6d5efb710a..93e37125eaa33 100644 --- a/python/pyspark/sql/tests/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -446,15 +446,16 @@ def int_index(pdf): def column_name_typo(pdf): return pd.DataFrame({'iid': pdf.id, 'v': pdf.v}) - @pandas_udf('id long, v int', PandasUDFType.GROUPED_MAP) + @pandas_udf('id long, v decimal', PandasUDFType.GROUPED_MAP) def invalid_positional_types(pdf): - return pd.DataFrame([(u'a', 1.2)]) + return pd.DataFrame([(1, datetime.date(2020, 10, 5))]) - with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, "KeyError: 'id'"): - grouped_df.apply(column_name_typo).collect() - with self.assertRaisesRegexp(Exception, "an integer is required"): - grouped_df.apply(invalid_positional_types).collect() + with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": False}): + with QuietTest(self.sc): + with self.assertRaisesRegexp(Exception, "KeyError: 'id'"): + grouped_df.apply(column_name_typo).collect() + with self.assertRaisesRegexp(Exception, "[D|d]ecimal.*got.*date"): + grouped_df.apply(invalid_positional_types).collect() def test_positional_assignment_conf(self): with self.sql_conf({ From b5e4b8c73e10743eef4d35b6e82053a5a065b2ed Mon Sep 17 00:00:00 2001 From: Michael Munday Date: Tue, 6 Oct 2020 08:31:06 -0500 Subject: [PATCH 47/59] [SPARK-27428][CORE][TEST] Increase receive buffer size used in StatsdSinkSuite ### What changes were proposed in this pull request? Increase size of socket receive buffer in these tests. ### Why are the changes needed? The socket receive buffer size set in this test was too small for the StatsdSinkSuite tests to run reliably on some systems. For a test in this suite to run reliably the buffer needs to be large enough to hold all the data in the packets being sent in a test along with any additional kernel or protocol overhead. The amount of kernel overhead per packet can vary from system to system but is typically far higher than the protocol overhead. If the receive buffer is too small and fills up then packets are silently dropped. This leads to the test failing with a timeout. If the socket defaults to a larger receive buffer (normally true) then we should keep that size. As well as increasing the minimum buffer size I've also decoupled the datagram packet buffer size from the receive buffer size. The receive buffer should in general be far larger to account for the fact that multiple packets might be buffered, as well as the aforementioned overhead. Any truncated data in individual packets will be picked up by the tests. ### Does this PR introduce _any_ user-facing change? No, this only affects the tests. ### How was this patch tested? Existing tests on IBM Z and x86. Closes #29819 from mundaym/fix-statsd. Authored-by: Michael Munday Signed-off-by: Sean Owen --- .../spark/metrics/sink/StatsdSinkSuite.scala | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala index 0e21a36071c42..3d4b8c868d6fc 100644 --- a/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala @@ -35,12 +35,27 @@ class StatsdSinkSuite extends SparkFunSuite { STATSD_KEY_UNIT -> "seconds", STATSD_KEY_HOST -> "127.0.0.1" ) - private val socketTimeout = 30000 // milliseconds - private val socketBufferSize = 8192 + // The maximum size of a single datagram packet payload. Payloads + // larger than this will be truncated. + private val maxPayloadSize = 256 // bytes + + // The receive buffer must be large enough to hold all inflight + // packets. This includes any kernel and protocol overhead. + // This value was determined experimentally and should be + // increased if timeouts are seen. + private val socketMinRecvBufferSize = 16384 // bytes + private val socketTimeout = 30000 // milliseconds private def withSocketAndSink(testCode: (DatagramSocket, StatsdSink) => Any): Unit = { val socket = new DatagramSocket - socket.setReceiveBufferSize(socketBufferSize) + + // Leave the receive buffer size untouched unless it is too + // small. If the receive buffer is too small packets will be + // silently dropped and receive operations will timeout. + if (socket.getReceiveBufferSize() < socketMinRecvBufferSize) { + socket.setReceiveBufferSize(socketMinRecvBufferSize) + } + socket.setSoTimeout(socketTimeout) val props = new Properties defaultProps.foreach(e => props.put(e._1, e._2)) @@ -61,7 +76,7 @@ class StatsdSinkSuite extends SparkFunSuite { sink.registry.register("counter", counter) sink.report() - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) socket.receive(p) val result = new String(p.getData, 0, p.getLength, UTF_8) @@ -77,7 +92,7 @@ class StatsdSinkSuite extends SparkFunSuite { sink.registry.register("gauge", gauge) sink.report() - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) socket.receive(p) val result = new String(p.getData, 0, p.getLength, UTF_8) @@ -87,7 +102,7 @@ class StatsdSinkSuite extends SparkFunSuite { test("metrics StatsD sink with Histogram") { withSocketAndSink { (socket, sink) => - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) val histogram = new Histogram(new UniformReservoir) histogram.update(10) histogram.update(20) @@ -121,7 +136,7 @@ class StatsdSinkSuite extends SparkFunSuite { test("metrics StatsD sink with Timer") { withSocketAndSink { (socket, sink) => - val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val p = new DatagramPacket(new Array[Byte](maxPayloadSize), maxPayloadSize) val timer = new Timer() timer.update(1, SECONDS) timer.update(2, SECONDS) From ec6fccb922f721e5a44d89c93f711f44ce9d6592 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 6 Oct 2020 14:33:34 +0000 Subject: [PATCH 48/59] [SPARK-32243][SQL][FOLLOWUP] Fix compilation in HiveSessionCatalog Fix a mistake when merging https://github.com/apache/spark/pull/29054 Closes #29955 from cloud-fan/hot-fix. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index f24834b938a1e..8a248a251820f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -126,7 +126,7 @@ private[sql] class HiveSessionCatalog( // Hive UDF/UDAF/UDTF with function definition. Otherwise, we just throw it earlier. case _: InvalidUDFClassException => makeHiveFunctionExpression(name, clazz, input) - case e => throw e + case NonFatal(e) => throw e } } } From 17d309dfacd4bdebbcd9609dd24a9e65a1a2b4f5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 6 Oct 2020 16:01:10 +0000 Subject: [PATCH 49/59] [SPARK-32963][SQL] empty string should be consistent for schema name in SparkGetSchemasOperation ### What changes were proposed in this pull request? This PR makes the empty string for schema name pattern match the global temp view as same as it works for other databases. This PR also add new tests to covering different kinds of wildcards to verify the SparkGetSchemasOperation ### Why are the changes needed? When the schema name is empty string, it is considered as ".*" and can match all databases in the catalog. But when it can not match the global temp view as it is not converted to ".*" ### Does this PR introduce _any_ user-facing change? yes , JDBC operation like `statement.getConnection.getMetaData..getSchemas(null, "")` now also provides the global temp view in the result set. ### How was this patch tested? new tests Closes #29834 from yaooqinn/SPARK-32963. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../SparkGetSchemasOperation.scala | 3 +- .../SparkMetadataOperationSuite.scala | 35 ++++++++++++++----- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 16fd502048e80..e58357a415545 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -77,7 +77,8 @@ private[hive] class SparkGetSchemasOperation( val globalTempViewDb = sqlContext.sessionState.catalog.globalTempViewManager.database val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) - if (databasePattern.matcher(globalTempViewDb).matches()) { + if (schemaName == null || schemaName.isEmpty || + databasePattern.matcher(globalTempViewDb).matches()) { rowSet.addRow(Array[AnyRef](globalTempViewDb, DEFAULT_HIVE_CATALOG)) } setState(OperationState.FINISHED) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index 7369dbfcf7a51..818f387f131d6 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{DatabaseMetaData, ResultSet} +import org.apache.hive.service.cli.HiveSQLException + import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.types._ @@ -28,23 +30,40 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { test("Spark's own GetSchemasOperation(SparkGetSchemasOperation)") { def checkResult(rs: ResultSet, dbNames: Seq[String]): Unit = { - for (i <- dbNames.indices) { - assert(rs.next()) - assert(rs.getString("TABLE_SCHEM") === dbNames(i)) + val expected = dbNames.iterator + while(rs.next() || expected.hasNext) { + assert(rs.getString("TABLE_SCHEM") === expected.next) + assert(rs.getString("TABLE_CATALOG").isEmpty) } // Make sure there are no more elements assert(!rs.next()) + assert(!expected.hasNext, "All expected schemas should be visited") } - withDatabase("db1", "db2") { statement => - Seq("CREATE DATABASE db1", "CREATE DATABASE db2").foreach(statement.execute) - + val dbs = Seq("db1", "db2", "db33", "db44") + val dbDflts = Seq("default", "global_temp") + withDatabase(dbs: _*) { statement => + dbs.foreach( db => statement.execute(s"CREATE DATABASE IF NOT EXISTS $db")) val metaData = statement.getConnection.getMetaData - checkResult(metaData.getSchemas(null, "%"), Seq("db1", "db2", "default", "global_temp")) + Seq("", "%", null, ".*", "_*", "_%", ".%") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs ++ dbDflts) + } + + Seq("db%", "db*") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs) + } + + Seq("db_", "db.") foreach { pattern => + checkResult(metaData.getSchemas(null, pattern), dbs.take(2)) + } + checkResult(metaData.getSchemas(null, "db1"), Seq("db1")) checkResult(metaData.getSchemas(null, "db_not_exist"), Seq.empty) - checkResult(metaData.getSchemas(null, "db*"), Seq("db1", "db2")) + + val e = intercept[HiveSQLException](metaData.getSchemas(null, "*")) + assert(e.getCause.getMessage === + "Error operating GET_SCHEMAS Dangling meta character '*' near index 0\n*\n^") } } From 3b2a38d73578e8760dbd6c34e427896a8cde00dd Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 6 Oct 2020 09:40:16 -0700 Subject: [PATCH 50/59] [SPARK-32511][SQL][FOLLOWUP] Fix the broken build for Scala 2.13 with Maven ### What changes were proposed in this pull request? This PR fixes the broken build for Scala 2.13 with Maven. https://github.com/apache/spark/pull/29913/checks?check_run_id=1187826966 #29795 was merged though it doesn't successfully finish the build for Scala 2.13 ### Why are the changes needed? To fix the build. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? `build/mvn -Pscala-2.13 -Phive -Phive-thriftserver -DskipTests package` Closes #29954 from sarutak/hotfix-seq. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/complexTypeCreator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index d5b1950e82c56..f6485a51f8fae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -586,7 +586,7 @@ case class WithField(name: String, valExpr: Expression) } } if (!hasMatch) result += newFieldExpr - result + result.toSeq } override def children: Seq[Expression] = valExpr :: Nil From 0b326d532752fd4e05b08dd16c096f80afe7d727 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 6 Oct 2020 14:18:37 -0700 Subject: [PATCH 51/59] [SPARK-32857][CORE] Fix flaky o.a.s.s.BarrierTaskContextSuite.throw exception if the number of barrier() calls are not the same on every task ### What changes were proposed in this pull request? Fix the flaky test. ### Why are the changes needed? The test is flaky: `Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown`. Check the full error stack [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport/org.apache.spark.scheduler/BarrierTaskContextSuite/throw_exception_if_the_number_of_barrier___calls_are_not_the_same_on_every_task/). By analyzing the log below, I found that task 0 hadn't reached the second `context.barrier()` when another three tasks already raised the sync timeout exceptions by the first `context.barrier()`. The timeout exceptions were caught by the `try...catch...`. Then, each task started another round barrier sync from the second `context.barrier()` and completed the sync successfully. ```scala 20/09/10 20:54:48.821 dispatcher-event-loop-10 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:48.822 dispatcher-event-loop-10 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 1/4. 20/09/10 20:54:48.826 dispatcher-BlockManagerMaster INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:38420 (size: 2.2 KiB, free: 546.3 MiB) 20/09/10 20:54:48.908 dispatcher-event-loop-12 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:48.909 dispatcher-event-loop-12 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 2/4. 20/09/10 20:54:48.959 dispatcher-event-loop-11 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:48.960 dispatcher-event-loop-11 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 3/4. 20/09/10 20:54:49.616 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0. 20/09/10 20:54:49.899 dispatcher-event-loop-15 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:49.900 dispatcher-event-loop-15 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 1, current progress: 1/4. 20/09/10 20:54:49.965 dispatcher-event-loop-13 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:49.966 dispatcher-event-loop-13 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 3, current progress: 2/4. 20/09/10 20:54:50.112 dispatcher-event-loop-16 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:50.113 dispatcher-event-loop-16 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 0, current progress: 3/4. 20/09/10 20:54:50.609 dispatcher-CoarseGrainedScheduler INFO TaskSchedulerImpl: Skip current round of resource offers for barrier stage 0 because the barrier taskSet requires 4 slots, while the total number of available slots is 0. 20/09/10 20:54:50.826 dispatcher-event-loop-17 INFO BarrierCoordinator: Current barrier epoch for Stage 0 (Attempt 0) is 0. 20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received update from Task 2, current progress: 4/4. 20/09/10 20:54:50.827 dispatcher-event-loop-17 INFO BarrierCoordinator: Barrier sync epoch 0 from Stage 0 (Attempt 0) received all updates from tasks, finished successfully. ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated the test and tested a hundred times without failure(Previously, there could be several failures). Closes #29732 from Ngone51/fix-flaky-throw-exception. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/scheduler/BarrierTaskContextSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index d18ca36f1fa60..e4ec62f8efc5b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -189,7 +189,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with test("throw exception if the number of barrier() calls are not the same on every task") { initLocalClusterSparkContext() - sc.conf.set("spark.barrier.sync.timeout", "1") + sc.conf.set("spark.barrier.sync.timeout", "5") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -212,7 +212,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with rdd2.collect() }.getMessage assert(error.contains("The coordinator didn't get all barrier sync requests")) - assert(error.contains("within 1 second(s)")) + assert(error.contains("within 5 second(s)")) } def testBarrierTaskKilled(interruptOnKill: Boolean): Unit = { From 57ed5a829b7dd8c92e5dfb7bb96373c8f464246c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 6 Oct 2020 16:59:23 -0700 Subject: [PATCH 52/59] [SPARK-33007][SQL] Simplify named_struct + get struct field + from_json expression chain ### What changes were proposed in this pull request? This proposes to simplify named_struct + get struct field + from_json expression chain from `struct(from_json.col1, from_json.col2, from_json.col3...)` to `struct(from_json)`. ### Why are the changes needed? Simplify complex expression tree that could be produced by query optimization or user. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test. Closes #29942 from viirya/SPARK-33007. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../optimizer/OptimizeJsonExprs.scala | 36 ++++++++++ .../optimizer/OptimizeJsonExprsSuite.scala | 67 +++++++++++++++++++ 2 files changed, 103 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala index 59228904d84b7..fcd5412d66d41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprs.scala @@ -28,10 +28,46 @@ import org.apache.spark.sql.types.{ArrayType, StructType} * The optimization includes: * 1. JsonToStructs(StructsToJson(child)) => child. * 2. Prune unnecessary columns from GetStructField/GetArrayStructFields + JsonToStructs. + * 3. CreateNamedStruct(JsonToStructs(json).col1, JsonToStructs(json).col2, ...) => + * If(IsNull(json), nullStruct, KnownNotNull(JsonToStructs(prunedSchema, ..., json))) + * if JsonToStructs(json) is shared among all fields of CreateNamedStruct. `prunedSchema` + * contains all accessed fields in original CreateNamedStruct. */ object OptimizeJsonExprs extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { case p => p.transformExpressions { + + case c: CreateNamedStruct + // If we create struct from various fields of the same `JsonToStructs`. + if c.valExprs.forall { v => + v.isInstanceOf[GetStructField] && + v.asInstanceOf[GetStructField].child.isInstanceOf[JsonToStructs] && + v.children.head.semanticEquals(c.valExprs.head.children.head) + } => + val jsonToStructs = c.valExprs.map(_.children.head) + val sameFieldName = c.names.zip(c.valExprs).forall { + case (name, valExpr: GetStructField) => + name.toString == valExpr.childSchema(valExpr.ordinal).name + case _ => false + } + + // Although `CreateNamedStruct` allows duplicated field names, e.g. "a int, a int", + // `JsonToStructs` does not support parsing json with duplicated field names. + val duplicateFields = c.names.map(_.toString).distinct.length != c.names.length + + // If we create struct from various fields of the same `JsonToStructs` and we don't + // alias field names and there is no duplicated field in the struct. + if (sameFieldName && !duplicateFields) { + val fromJson = jsonToStructs.head.asInstanceOf[JsonToStructs].copy(schema = c.dataType) + val nullFields = c.children.grouped(2).flatMap { + case Seq(name, value) => Seq(name, Literal(null, value.dataType)) + }.toSeq + + If(IsNull(fromJson.child), c.copy(children = nullFields), KnownNotNull(fromJson)) + } else { + c + } + case jsonToStructs @ JsonToStructs(_, options1, StructsToJson(options2, child, timeZoneId2), timeZoneId1) if options1.isEmpty && options2.isEmpty && timeZoneId1 == timeZoneId2 && diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala index e47a141dfed1f..7d975a1b00466 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala @@ -199,4 +199,71 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { JsonToStructs(prunedSchema2, options, 'json), field2, 0, 1, false).as("b")).analyze comparePlans(optimized2, expected2) } + + test("SPARK-33007: simplify named_struct + from_json") { + val options = Map.empty[String, String] + val schema = StructType.fromDDL("a int, b int, c long, d string") + + val prunedSchema1 = StructType.fromDDL("a int, b int") + val nullStruct = namedStruct("a", Literal(null, IntegerType), "b", Literal(null, IntegerType)) + + val UTC_OPT = Option("UTC") + val json: BoundReference = 'json.string.canBeNull.at(0) + + assertEquivalent( + testRelation2, + namedStruct( + "a", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 0), + "b", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 1)).as("struct"), + If(IsNull(json), + nullStruct, + KnownNotNull(JsonToStructs(prunedSchema1, options, json, UTC_OPT))).as("struct")) + + val field1 = StructType.fromDDL("a int") + val field2 = StructType.fromDDL("b int") + + // Skip optimization if `namedStruct` aliases field name. + assertEquivalent( + testRelation2, + namedStruct( + "a1", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 0), + "b", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 1)).as("struct"), + namedStruct( + "a1", GetStructField(JsonToStructs(field1, options, json, UTC_OPT), 0), + "b", GetStructField(JsonToStructs(field2, options, json, UTC_OPT), 0)).as("struct")) + + assertEquivalent( + testRelation2, + namedStruct( + "a", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 0), + "a", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 0)).as("struct"), + namedStruct( + "a", GetStructField(JsonToStructs(field1, options, json, UTC_OPT), 0), + "a", GetStructField(JsonToStructs(field1, options, json, UTC_OPT), 0)).as("struct")) + + val PST = getZoneId("-08:00") + // Skip optimization if `JsonToStructs`s are not the same. + assertEquivalent( + testRelation2, + namedStruct( + "a", GetStructField(JsonToStructs(schema, options, json, UTC_OPT), 0), + "b", GetStructField(JsonToStructs(schema, options, json, Option(PST.getId)), 1)) + .as("struct"), + namedStruct( + "a", GetStructField(JsonToStructs(field1, options, json, UTC_OPT), 0), + "b", GetStructField(JsonToStructs(field2, options, json, Option(PST.getId)), 0)) + .as("struct")) + } + + private def assertEquivalent(relation: LocalRelation, e1: Expression, e2: Expression): Unit = { + val plan = relation.select(e1).analyze + val actual = Optimizer.execute(plan) + val expected = relation.select(e2).analyze + comparePlans(actual, expected) + + Seq("""{"a":1, "b":2, "c": 123, "d": "test"}""", null).foreach(v => { + val row = create_row(v) + checkEvaluation(e1, e2.eval(row), row) + }) + } } From 584f90c82e8e47cdcaab50f95e6c709f460cd789 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 7 Oct 2020 09:29:30 +0900 Subject: [PATCH 53/59] [SPARK-33067][SQL][TESTS][FOLLOWUP] Check error messages in JDBCTableCatalogSuite ### What changes were proposed in this pull request? Get error message from the expected exception, and check that they are reasonable. ### Why are the changes needed? To improve tests by expecting particular error messages. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running `JDBCTableCatalogSuite`. Closes #29957 from MaxGekk/jdbcv2-negative-tests-followup. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../v2/jdbc/JDBCTableCatalogSuite.scala | 116 +++++++++++------- 1 file changed, 71 insertions(+), 45 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index bf71f90779b71..ca86a8f593621 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -22,6 +22,7 @@ import java.util.Properties import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -75,10 +76,14 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "to_drop"), Row("test", "people"))) sql("DROP TABLE h2.test.to_drop") checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people"))) - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[NoSuchTableException] { + Seq( + "h2.test.not_existing_table" -> "Table test.not_existing_table not found", + "h2.bad_test.not_existing_table" -> "Table bad_test.not_existing_table not found" + ).foreach { case (table, expectedMsg) => + val msg = intercept[NoSuchTableException] { sql(s"DROP TABLE $table") - } + }.getMessage + assert(msg.contains(expectedMsg)) } } @@ -96,10 +101,14 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { Seq(Row("test", "dst_table"), Row("test", "people"))) } // Rename not existing table or namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[org.h2.jdbc.JdbcSQLException] { + Seq( + "h2.test.not_existing_table" -> "Table \"not_existing_table\" not found", + "h2.bad_test.not_existing_table" -> "Schema \"bad_test\" not found" + ).foreach { case (table, expectedMsg) => + val msg = intercept[org.h2.jdbc.JdbcSQLException] { sql(s"ALTER TABLE $table RENAME TO test.dst_table") - } + }.getMessage + assert(msg.contains(expectedMsg)) } // Rename to an existing table withTable("h2.test.dst_table") { @@ -110,9 +119,10 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { withConnection { conn => conn.prepareStatement("""CREATE TABLE "test"."src_table" (id INTEGER)""").executeUpdate() } - intercept[org.h2.jdbc.JdbcSQLException] { - sql("ALTER TABLE h2.test.src_table RENAME TO h2.test.dst_table") - } + val msg = intercept[org.h2.jdbc.JdbcSQLException] { + sql("ALTER TABLE h2.test.src_table RENAME TO test.dst_table") + }.getMessage + assert(msg.contains("Table \"dst_table\" already exists")) } } } @@ -124,9 +134,10 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("ID", IntegerType) assert(t.schema === expectedSchema) Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { - spark.table(s"h2.$table").schema - } + val msg = intercept[AnalysisException] { + spark.table(table).schema + }.getMessage + assert(msg.contains("Table or view not found")) } } @@ -140,13 +151,15 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { } withTable("h2.test.new_table") { sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql("CREATE TABLE h2.test.new_table(i INT, j STRING) USING _") - } + }.getMessage + assert(msg.contains("Table test.new_table already exists")) } - intercept[org.h2.jdbc.JdbcSQLException] { + val msg = intercept[org.h2.jdbc.JdbcSQLException] { sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING) USING _") - } + }.getMessage + assert(msg.contains("Schema \"bad_test\" not found")) } test("alter table ... add column") { @@ -164,15 +177,17 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { expectedSchema = expectedSchema.add("C3", DoubleType) assert(t.schema === expectedSchema) // Add already existing column - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table ADD COLUMNS (C3 DOUBLE)") - } + }.getMessage + assert(msg.contains("Cannot add column, because C3 already exists")) } // Add a column to not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table ADD COLUMNS (C4 STRING)") - } + }.getMessage + assert(msg.contains("Table not found")) } } @@ -186,15 +201,17 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("C0", IntegerType) assert(t.schema === expectedSchema) // Rename to already existing column - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table RENAME COLUMN C TO C0") - } + }.getMessage + assert(msg.contains("Cannot rename column, because C0 already exists")) } // Rename a column in not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table RENAME COLUMN ID TO C") - } + }.getMessage + assert(msg.contains("Table not found")) } } @@ -206,15 +223,17 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val expectedSchema = new StructType().add("C2", IntegerType) assert(t.schema === expectedSchema) // Drop not existing column - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table DROP COLUMN bad_column") - } + }.getMessage + assert(msg.contains("Cannot delete missing field bad_column in test.alt_table schema")) } // Drop a column to not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table DROP COLUMN C1") - } + }.getMessage + assert(msg.contains("Table not found")) } } @@ -226,19 +245,22 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val expectedSchema = new StructType().add("ID", DoubleType) assert(t.schema === expectedSchema) // Update not existing column - intercept[AnalysisException] { + val msg1 = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column TYPE DOUBLE") - } + }.getMessage + assert(msg1.contains("Cannot update missing field bad_column in test.alt_table schema")) // Update column to wrong type - intercept[AnalysisException] { + val msg2 = intercept[ParseException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN id TYPE bad_type") - } + }.getMessage + assert(msg2.contains("DataType bad_type is not supported")) } // Update column type in not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN id TYPE DOUBLE") - } + }.getMessage + assert(msg.contains("Table not found")) } } @@ -250,35 +272,39 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val expectedSchema = new StructType().add("ID", IntegerType, nullable = true) assert(t.schema === expectedSchema) // Update nullability of not existing column - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column DROP NOT NULL") - } + }.getMessage + assert(msg.contains("Cannot update missing field bad_column in test.alt_table")) } // Update column nullability in not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN ID DROP NOT NULL") - } + }.getMessage + assert(msg.contains("Table not found")) } } test("alter table ... update column comment not supported") { withTable("h2.test.alt_table") { sql("CREATE TABLE h2.test.alt_table (ID INTEGER) USING _") - val thrown = intercept[java.sql.SQLFeatureNotSupportedException] { + val msg1 = intercept[java.sql.SQLFeatureNotSupportedException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN ID COMMENT 'test'") - } - assert(thrown.getMessage.contains("Unsupported TableChange")) + }.getMessage + assert(msg1.contains("Unsupported TableChange")) // Update comment for not existing column - intercept[AnalysisException] { + val msg2 = intercept[AnalysisException] { sql("ALTER TABLE h2.test.alt_table ALTER COLUMN bad_column COMMENT 'test'") - } + }.getMessage + assert(msg2.contains("Cannot update missing field bad_column in test.alt_table")) } // Update column comments in not existing table and namespace Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - intercept[AnalysisException] { + val msg = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN ID COMMENT 'test'") - } + }.getMessage + assert(msg.contains("Table not found")) } } } From 5ce321dc80a699fa525ca5b69bf2c28e10f8a12a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 7 Oct 2020 13:00:59 +0900 Subject: [PATCH 54/59] [SPARK-33017][PYTHON][DOCS][FOLLOW-UP] Add getCheckpointDir into API documentation ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/29918. We should add it into the documentation as well. ### Why are the changes needed? To show users new APIs. ### Does this PR introduce _any_ user-facing change? Yes, `SparkContext.getCheckpointDir` will be documented. ### How was this patch tested? Manually built the PySpark documentation: ```bash cd python/docs make clean html cd build/html open index.html ``` Closes #29960 from HyukjinKwon/SPARK-33017. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/docs/source/reference/pyspark.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/python/docs/source/reference/pyspark.rst b/python/docs/source/reference/pyspark.rst index c13df6ee2d2b4..fc0775eb7f8f5 100644 --- a/python/docs/source/reference/pyspark.rst +++ b/python/docs/source/reference/pyspark.rst @@ -64,6 +64,7 @@ Spark Context APIs SparkContext.defaultParallelism SparkContext.dump_profiles SparkContext.emptyRDD + SparkContext.getCheckpointDir SparkContext.getConf SparkContext.getLocalProperty SparkContext.getOrCreate From aea78d2c8cdf12f4978fa6a69107d096c07c6fec Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 7 Oct 2020 04:48:57 +0000 Subject: [PATCH 55/59] [SPARK-33034][SQL] Support ALTER TABLE in JDBC v2 Table Catalog: add, update type and nullability of columns (Oracle dialect) ### What changes were proposed in this pull request? 1. Override the default SQL strings in the Oracle Dialect for: - ALTER TABLE ADD COLUMN - ALTER TABLE UPDATE COLUMN TYPE - ALTER TABLE UPDATE COLUMN NULLABILITY 2. Add new docker integration test suite `jdbc/v2/OracleIntegrationSuite.scala` ### Why are the changes needed? In SPARK-24907, we implemented JDBC v2 Table Catalog but it doesn't support some `ALTER TABLE` at the moment. This PR supports Oracle specific `ALTER TABLE`. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By running new integration test suite: ``` $ ./build/sbt -Pdocker-integration-tests "test-only *.OracleIntegrationSuite" ``` Closes #29912 from MaxGekk/jdbcv2-oracle-alter-table. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/jdbc/DockerJDBCIntegrationSuite.scala | 15 +- .../sql/jdbc/v2/OracleIntegrationSuite.scala | 152 ++++++++++++++++++ .../datasources/v2/jdbc/JDBCTable.scala | 1 - .../apache/spark/sql/jdbc/JdbcDialects.scala | 25 ++- .../apache/spark/sql/jdbc/OracleDialect.scala | 19 +++ 5 files changed, 200 insertions(+), 12 deletions(-) create mode 100644 external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index 6d1a22dd22b65..24927da16d50c 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -98,7 +98,13 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu val connectionTimeout = timeout(2.minutes) private var docker: DockerClient = _ - protected var externalPort: Int = _ + // Configure networking (necessary for boot2docker / Docker Machine) + protected lazy val externalPort: Int = { + val sock = new ServerSocket(0) + val port = sock.getLocalPort + sock.close() + port + } private var containerId: String = _ protected var jdbcUrl: String = _ @@ -122,13 +128,6 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu log.warn(s"Docker image ${db.imageName} not found; pulling image from registry") docker.pull(db.imageName) } - // Configure networking (necessary for boot2docker / Docker Machine) - externalPort = { - val sock = new ServerSocket(0) - val port = sock.getLocalPort - sock.close() - port - } val hostConfigBuilder = HostConfig.builder() .privileged(db.privileged) .networkMode("bridge") diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala new file mode 100644 index 0000000000000..400459c0ea17b --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala @@ -0,0 +1,152 @@ +/* + * 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.spark.sql.jdbc.v2 + +import java.sql.Connection + +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkConf +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog +import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ +import org.apache.spark.tags.DockerTest + +/** + * The following would be the steps to test this + * 1. Build Oracle database in Docker, please refer below link about how to. + * https://github.com/oracle/docker-images/blob/master/OracleDatabase/SingleInstance/README.md + * 2. export ORACLE_DOCKER_IMAGE_NAME=$ORACLE_DOCKER_IMAGE_NAME + * Pull oracle $ORACLE_DOCKER_IMAGE_NAME image - docker pull $ORACLE_DOCKER_IMAGE_NAME + * 3. Start docker - sudo service docker start + * 4. Run spark test - ./build/sbt -Pdocker-integration-tests + * "test-only org.apache.spark.sql.jdbc.v2.OracleIntegrationSuite" + * + * An actual sequence of commands to run the test is as follows + * + * $ git clone https://github.com/oracle/docker-images.git + * // Head SHA: 3e352a22618070595f823977a0fd1a3a8071a83c + * $ cd docker-images/OracleDatabase/SingleInstance/dockerfiles + * $ ./buildDockerImage.sh -v 18.4.0 -x + * $ export ORACLE_DOCKER_IMAGE_NAME=oracle/database:18.4.0-xe + * $ cd $SPARK_HOME + * $ ./build/sbt -Pdocker-integration-tests + * "test-only org.apache.spark.sql.jdbc.v2.OracleIntegrationSuite" + * + * It has been validated with 18.4.0 Express Edition. + */ +@DockerTest +class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSparkSession { + override val db = new DatabaseOnDocker { + override val imageName = sys.env("ORACLE_DOCKER_IMAGE_NAME") + override val env = Map( + "ORACLE_PWD" -> "oracle" + ) + override val usesIpc = false + override val jdbcPort: Int = 1521 + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:oracle:thin:system/oracle@//$ip:$port/xe" + } + + override def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.catalog.oracle", classOf[JDBCTableCatalog].getName) + .set("spark.sql.catalog.oracle.url", db.getJdbcUrl(dockerIp, externalPort)) + + override val connectionTimeout = timeout(7.minutes) + override def dataPreparation(conn: Connection): Unit = {} + + test("SPARK-33034: ALTER TABLE ... add new columns") { + withTable("oracle.alt_table") { + sql("CREATE TABLE oracle.alt_table (ID STRING) USING _") + sql("ALTER TABLE oracle.alt_table ADD COLUMNS (C1 STRING, C2 STRING)") + var t = spark.table("oracle.alt_table") + var expectedSchema = new StructType() + .add("ID", StringType) + .add("C1", StringType) + .add("C2", StringType) + assert(t.schema === expectedSchema) + sql("ALTER TABLE oracle.alt_table ADD COLUMNS (C3 STRING)") + t = spark.table("oracle.alt_table") + expectedSchema = expectedSchema.add("C3", StringType) + assert(t.schema === expectedSchema) + // Add already existing column + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE oracle.alt_table ADD COLUMNS (C3 DOUBLE)") + }.getMessage + assert(msg.contains("Cannot add column, because C3 already exists")) + } + // Add a column to not existing table + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE oracle.not_existing_table ADD COLUMNS (C4 STRING)") + }.getMessage + assert(msg.contains("Table not found")) + } + + test("SPARK-33034: ALTER TABLE ... update column type") { + withTable("oracle.alt_table") { + sql("CREATE TABLE oracle.alt_table (ID INTEGER) USING _") + sql("ALTER TABLE oracle.alt_table ALTER COLUMN id TYPE STRING") + val t = spark.table("oracle.alt_table") + val expectedSchema = new StructType().add("ID", StringType) + assert(t.schema === expectedSchema) + // Update column type from STRING to INTEGER + val msg1 = intercept[AnalysisException] { + sql("ALTER TABLE oracle.alt_table ALTER COLUMN id TYPE INTEGER") + }.getMessage + assert(msg1.contains("Cannot update alt_table field ID: string cannot be cast to int")) + // Update not existing column + val msg2 = intercept[AnalysisException] { + sql("ALTER TABLE oracle.alt_table ALTER COLUMN bad_column TYPE DOUBLE") + }.getMessage + assert(msg2.contains("Cannot update missing field bad_column")) + // Update column to wrong type + val msg3 = intercept[ParseException] { + sql("ALTER TABLE oracle.alt_table ALTER COLUMN id TYPE bad_type") + }.getMessage + assert(msg3.contains("DataType bad_type is not supported")) + } + // Update column type in not existing table + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE oracle.not_existing_table ALTER COLUMN id TYPE DOUBLE") + }.getMessage + assert(msg.contains("Table not found")) + } + + test("SPARK-33034: ALTER TABLE ... update column nullability") { + withTable("oracle.alt_table") { + sql("CREATE TABLE oracle.alt_table (ID STRING NOT NULL) USING _") + sql("ALTER TABLE oracle.alt_table ALTER COLUMN ID DROP NOT NULL") + val t = spark.table("oracle.alt_table") + val expectedSchema = new StructType().add("ID", StringType, nullable = true) + assert(t.schema === expectedSchema) + // Update nullability of not existing column + val msg = intercept[AnalysisException] { + sql("ALTER TABLE oracle.alt_table ALTER COLUMN bad_column DROP NOT NULL") + }.getMessage + assert(msg.contains("Cannot update missing field bad_column")) + } + // Update column nullability in not existing table + val msg = intercept[AnalysisException] { + sql(s"ALTER TABLE oracle.not_existing_table ALTER COLUMN ID DROP NOT NULL") + }.getMessage + assert(msg.contains("Table not found")) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala index 55759497bd910..5e11ea66be4c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions) extends Table with SupportsRead with SupportsWrite { - assert(ident.namespace().length == 1) override def name(): String = ident.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index cea5a20917532..a01720d1eefc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -212,7 +212,7 @@ abstract class JdbcDialect extends Serializable { case add: AddColumn if add.fieldNames.length == 1 => val dataType = JdbcUtils.getJdbcType(add.dataType(), this).databaseTypeDefinition val name = add.fieldNames - updateClause += s"ALTER TABLE $tableName ADD COLUMN ${name(0)} $dataType" + updateClause += getAddColumnQuery(tableName, name(0), dataType) case rename: RenameColumn if rename.fieldNames.length == 1 => val name = rename.fieldNames updateClause += s"ALTER TABLE $tableName RENAME COLUMN ${name(0)} TO ${rename.newName}" @@ -223,17 +223,36 @@ abstract class JdbcDialect extends Serializable { val name = updateColumnType.fieldNames val dataType = JdbcUtils.getJdbcType(updateColumnType.newDataType(), this) .databaseTypeDefinition - updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} $dataType" + updateClause += getUpdateColumnTypeQuery(tableName, name(0), dataType) case updateNull: UpdateColumnNullability if updateNull.fieldNames.length == 1 => val name = updateNull.fieldNames val nullable = if (updateNull.nullable()) "NULL" else "NOT NULL" - updateClause += s"ALTER TABLE $tableName ALTER COLUMN ${name(0)} SET $nullable" + updateClause += getUpdateColumnNullabilityQuery(tableName, name(0), updateNull.nullable()) case _ => throw new SQLFeatureNotSupportedException(s"Unsupported TableChange $change") } } updateClause.result() } + + def getAddColumnQuery(tableName: String, columnName: String, dataType: String): String = { + s"ALTER TABLE $tableName ADD COLUMN $columnName $dataType" + } + + def getUpdateColumnTypeQuery( + tableName: String, + columnName: String, + newDataType: String): String = { + s"ALTER TABLE $tableName ALTER COLUMN $columnName $newDataType" + } + + def getUpdateColumnNullabilityQuery( + tableName: String, + columnName: String, + isNullable: Boolean): String = { + val nullable = if (isNullable) "NULL" else "NOT NULL" + s"ALTER TABLE $tableName ALTER COLUMN $columnName SET $nullable" + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 3f12b9acd0fc4..128b90a190481 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -118,4 +118,23 @@ private case object OracleDialect extends JdbcDialect { case _ => s"TRUNCATE TABLE $table" } } + + // see https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_3001.htm#SQLRF01001 + override def getAddColumnQuery(tableName: String, columnName: String, dataType: String): String = + s"ALTER TABLE $tableName ADD $columnName $dataType" + + // see https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_3001.htm#SQLRF01001 + override def getUpdateColumnTypeQuery( + tableName: String, + columnName: String, + newDataType: String): String = + s"ALTER TABLE $tableName MODIFY $columnName $newDataType" + + override def getUpdateColumnNullabilityQuery( + tableName: String, + columnName: String, + isNullable: Boolean): String = { + val nullable = if (isNullable) "NULL" else "NOT NULL" + s"ALTER TABLE $tableName MODIFY $columnName $nullable" + } } From 7e99fcd64efa425f3c985df4fe957a3be274a49a Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 7 Oct 2020 06:33:20 +0000 Subject: [PATCH 56/59] [SPARK-33004][SQL] Migrate DESCRIBE column to use UnresolvedTableOrView to resolve the identifier ### What changes were proposed in this pull request? This PR proposes to migrate `DESCRIBE tbl colname` to use `UnresolvedTableOrView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing). ### Why are the changes needed? The current behavior is not consistent between v1 and v2 commands when resolving a temp view. In v2, the `t` in the following example is resolved to a table: ```scala sql("CREATE TABLE testcat.ns.t (id bigint) USING foo") sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i") sql("USE testcat.ns") sql("DESCRIBE t i") // 't' is resolved to testcat.ns.t Describing columns is not supported for v2 tables.; org.apache.spark.sql.AnalysisException: Describing columns is not supported for v2 tables.; ``` whereas in v1, the `t` is resolved to a temp view: ```scala sql("CREATE DATABASE test") sql("CREATE TABLE spark_catalog.test.t (id bigint) USING csv") sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i") sql("USE spark_catalog.test") sql("DESCRIBE t i").show // 't' is resolved to a temp view +---------+----------+ |info_name|info_value| +---------+----------+ | col_name| i| |data_type| int| | comment| NULL| +---------+----------+ ``` ### Does this PR introduce _any_ user-facing change? After this PR, `DESCRIBE t i` is resolved to a temp view `t` instead of `testcat.ns.t`. ### How was this patch tested? Added a new test Closes #29880 from imback82/describe_column_consistent. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../catalyst/analysis/ResolveCatalogs.scala | 4 --- .../sql/catalyst/parser/AstBuilder.scala | 6 ++-- ...hema.scala => DescribeCommandSchema.scala} | 10 +++++-- .../catalyst/plans/logical/statements.scala | 8 ------ .../catalyst/plans/logical/v2Commands.scala | 15 ++++++++-- .../sql/catalyst/parser/DDLParserSuite.scala | 28 +++++++++---------- .../analysis/ResolveSessionCatalog.scala | 12 +++++--- .../spark/sql/execution/command/tables.scala | 13 ++------- .../datasources/v2/DataSourceV2Strategy.scala | 3 ++ .../apache/spark/sql/SQLQueryTestSuite.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 24 ++++++++++++++++ .../hive/execution/HiveComparisonTest.scala | 2 +- 12 files changed, 78 insertions(+), 49 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/{DescribeTableSchema.scala => DescribeCommandSchema.scala} (74%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 0d0f80be359e7..65ddff8c44ed9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -142,10 +142,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier) - case DescribeColumnStatement( - NonSessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => - throw new AnalysisException("Describing columns is not supported for v2 tables.") - case c @ CreateTableStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => assertNoNullTypeInSchema(c.tableSchema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f133235a2636e..f29e7b11e02de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3183,7 +3183,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create a [[DescribeColumnStatement]] or [[DescribeRelation]] commands. + * Create a [[DescribeColumn]] or [[DescribeRelation]] commands. */ override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null @@ -3191,8 +3191,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.partitionSpec != null) { throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) } else { - DescribeColumnStatement( - visitMultipartIdentifier(ctx.multipartIdentifier()), + DescribeColumn( + UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())), ctx.describeColName.nameParts.asScala.map(_.getText).toSeq, isExtended) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala similarity index 74% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala index ff35972b901f9..99d2ea7751959 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeCommandSchema.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} +import org.apache.spark.sql.types.{MetadataBuilder, StringType} -private[sql] object DescribeTableSchema { +private[sql] object DescribeCommandSchema { def describeTableAttributes(): Seq[AttributeReference] = Seq( AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), @@ -28,4 +28,10 @@ private[sql] object DescribeTableSchema { new MetadataBuilder().putString("comment", "data type of the column").build())(), AttributeReference("comment", StringType, nullable = true, new MetadataBuilder().putString("comment", "comment of the column").build())()) + + def describeColumnAttributes(): Seq[AttributeReference] = Seq( + AttributeReference("info_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column info").build())(), + AttributeReference("info_value", StringType, nullable = false, + new MetadataBuilder().putString("comment", "value of the column info").build())()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index d09e08d105c21..d7c097af9120f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -298,14 +298,6 @@ case class DropViewStatement( viewName: Seq[String], ifExists: Boolean) extends ParsedStatement -/** - * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. - */ -case class DescribeColumnStatement( - tableName: Seq[String], - colNameParts: Seq[String], - isExtended: Boolean) extends ParsedStatement - /** * An INSERT INTO statement, as parsed from SQL. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 475eb7d74773d..50af16ca276e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.{NamedRelation, UnresolvedException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable} -import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform @@ -312,7 +312,18 @@ case class DescribeRelation( partitionSpec: TablePartitionSpec, isExtended: Boolean) extends Command { override def children: Seq[LogicalPlan] = Seq(relation) - override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() + override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() +} + +/** + * The logical plan of the DESCRIBE relation_name col_name command that works for v2 tables. + */ +case class DescribeColumn( + relation: LogicalPlan, + colNameParts: Seq[String], + isExtended: Boolean) extends Command { + override def children: Seq[LogicalPlan] = Seq(relation) + override def output: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes() } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 378026b1ce9c6..8b8531b2bb3b1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -780,27 +780,27 @@ class DDLParserSuite extends AnalysisTest { test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = false)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `abc.xyz`"), - DescribeColumnStatement( - Seq("t"), Seq("abc.xyz"), isExtended = false)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("abc.xyz"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t abc.xyz"), - DescribeColumnStatement( - Seq("t"), Seq("abc", "xyz"), isExtended = false)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("abc", "xyz"), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"), - DescribeColumnStatement( - Seq("t"), Seq("a.b", "x.y"), isExtended = false)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("a.b", "x.y"), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE t col"), - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = false)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"), - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = true)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true)) comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"), - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = true)) + DescribeColumn( + UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true)) val caught = intercept[AnalysisException]( parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 11493ad59a760..24382e07a2966 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -256,16 +256,20 @@ class ResolveSessionCatalog( case RenameTableStatement(TempViewOrV1Table(oldName), newName, isView) => AlterTableRenameCommand(oldName.asTableIdentifier, newName.asTableIdentifier, isView) - case DescribeRelation(ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) => + case DescribeRelation(r @ ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) + if isSessionCatalog(r.catalog) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) - case DescribeColumnStatement(tbl, colNameParts, isExtended) => - val name = parseTempViewOrV1Table(tbl, "Describing columns") - DescribeColumnCommand(name.asTableIdentifier, colNameParts, isExtended) + case DescribeColumn(r @ ResolvedTable(_, _, _: V1Table), colNameParts, isExtended) + if isSessionCatalog(r.catalog) => + DescribeColumnCommand(r.identifier.asTableIdentifier, colNameParts, isExtended) + + case DescribeColumn(ResolvedView(ident), colNameParts, isExtended) => + DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended) // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the // session catalog and the table provider is not v2. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index e4be2a8d3bb8e..206f952fed0ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier, CaseInsensitiveMap} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} @@ -597,7 +597,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output = DescribeTableSchema.describeTableAttributes() + override val output = DescribeCommandSchema.describeTableAttributes() protected def describeSchema( schema: StructType, @@ -760,14 +760,7 @@ case class DescribeColumnCommand( isExtended: Boolean) extends RunnableCommand { - override val output: Seq[Attribute] = { - Seq( - AttributeReference("info_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column info").build())(), - AttributeReference("info_value", StringType, nullable = false, - new MetadataBuilder().putString("comment", "value of the column info").build())() - ) - } + override val output: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes() override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index c5ddba43a56aa..3841bd0a66987 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -225,6 +225,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } DescribeTableExec(desc.output, r.table, isExtended) :: Nil + case DescribeColumn(_: ResolvedTable, _, _) => + throw new AnalysisException("Describing columns is not supported for v2 tables.") + case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index b7cf0798a9d4b..0bb1f5e20fc5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -502,7 +502,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper case _: DescribeCommandBase | _: DescribeColumnCommand | _: DescribeRelation - | _: DescribeColumnStatement => true + | _: DescribeColumn => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e3782c7409198..e3618f1326941 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -168,7 +168,31 @@ class DataSourceV2SQLSuite Array("Provider", "foo", ""), Array(TableCatalog.PROP_OWNER.capitalize, defaultUser, ""), Array("Table Properties", "[bar=baz]", ""))) + } + test("Describe column is not supported for v2 catalog") { + withTable("testcat.tbl") { + spark.sql("CREATE TABLE testcat.tbl (id bigint) USING foo") + val ex = intercept[AnalysisException] { + spark.sql("DESCRIBE testcat.tbl id") + } + assert(ex.message.contains("Describing columns is not supported for v2 tables")) + } + } + + test("SPARK-33004: Describe column should resolve to a temporary view first") { + withTable("testcat.ns.t") { + withTempView("t") { + sql("CREATE TABLE testcat.ns.t (id bigint) USING foo") + sql("CREATE TEMPORARY VIEW t AS SELECT 2 as i") + sql("USE testcat.ns") + checkAnswer( + sql("DESCRIBE t i"), + Seq(Row("col_name", "i"), + Row("data_type", "int"), + Row("comment", "NULL"))) + } + } } test("CreateTable: use v2 plan and session catalog when provider is v2") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 7f198632a1cd6..01cf214574eeb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -375,7 +375,7 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && - (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive From 4e1ded67f88ffc869379319758d923aa538554b2 Mon Sep 17 00:00:00 2001 From: itholic Date: Wed, 7 Oct 2020 16:39:25 +0900 Subject: [PATCH 57/59] [SPARK-32189][DOCS][PYTHON][FOLLOW-UP] Fixed broken link and typo in PySpark docs ### What changes were proposed in this pull request? This PR is a follow-up of #29781 to fix broken link and typo. Screen Shot 2020-10-07 at 3 56 28 PM Screen Shot 2020-10-07 at 3 55 36 PM ### Why are the changes needed? Current link is not working properly because of wrong path. ### Does this PR introduce _any_ user-facing change? Yes, the link is working properly now. ### How was this patch tested? Manually built the doc. Closes #29963 from itholic/SPARK-32189-FOLLOWUP. Authored-by: itholic Signed-off-by: HyukjinKwon --- python/docs/source/development/debugging.rst | 2 +- python/docs/source/development/setting_ide.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/docs/source/development/debugging.rst b/python/docs/source/development/debugging.rst index c5f3351527f11..bc141a6f44a6f 100644 --- a/python/docs/source/development/debugging.rst +++ b/python/docs/source/development/debugging.rst @@ -35,7 +35,7 @@ with JVM. Profiling and debugging JVM is described at `Useful Developer Tools `__. +- If you are running locally, you can directly debug the driver side via using your IDE without the remote debug feature. Setting PySpark with IDEs is documented `here `__. - *There are many other ways of debugging PySpark applications*. For example, you can remotely debug by using the open source `Remote Debugger `_ instead of using PyCharm Professional documented here. diff --git a/python/docs/source/development/setting_ide.rst b/python/docs/source/development/setting_ide.rst index dcb44c1483006..6e8f0148c6eb3 100644 --- a/python/docs/source/development/setting_ide.rst +++ b/python/docs/source/development/setting_ide.rst @@ -50,7 +50,7 @@ Let's go to the path ``python/pyspark/tests`` in PyCharm and try to run the any You might can see the ``KeyError: 'SPARK_HOME'`` because the environment variable has not been set yet. Go **Run -> Edit Configurations**, and set the environment variables as below. -Please make sure to specify your own path for ``SPARK_HOME`` rather than ``/.../spark``. After completing the variable, click **Okay** to apply the changes. +Please make sure to specify your own path for ``SPARK_HOME`` rather than ``/.../spark``. After completing the variable, click **OK** to apply the changes. .. image:: ../../../../docs/img/pycharm-with-pyspark2.png :alt: Setting up SPARK_HOME From 72da6f86cfbdd36dac3fc440c333bc1db1935edd Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 7 Oct 2020 19:53:59 +0900 Subject: [PATCH 58/59] [SPARK-33002][PYTHON] Remove non-API annotations ### What changes were proposed in this pull request? This PR: - removes annotations for modules which are not part of the public API. - removes `__init__.pyi` files, if no annotations, beyond exports, are present. ### Why are the changes needed? Primarily to reduce maintenance overhead and as requested in the comments to https://github.com/apache/spark/pull/29591 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests and additional MyPy checks: ``` mypy --no-incremental --config python/mypy.ini python/pyspark MYPYPATH=python/ mypy --no-incremental --config python/mypy.ini examples/src/main/python/ml examples/src/main/python/sql examples/src/main/python/sql/streaming ``` Closes #29879 from zero323/SPARK-33002. Authored-by: zero323 Signed-off-by: HyukjinKwon --- python/mypy.ini | 5 +- python/pyspark/_globals.pyi | 27 ----- python/pyspark/accumulators.pyi | 4 +- python/pyspark/broadcast.pyi | 4 +- python/pyspark/daemon.pyi | 29 ----- python/pyspark/find_spark_home.pyi | 17 --- python/pyspark/java_gateway.pyi | 24 ----- python/pyspark/join.pyi | 50 --------- python/pyspark/ml/__init__.pyi | 45 -------- python/pyspark/mllib/__init__.pyi | 32 ------ python/pyspark/rddsampler.pyi | 54 ---------- python/pyspark/resource/__init__.pyi | 31 ------ python/pyspark/serializers.py | 2 +- python/pyspark/serializers.pyi | 122 ---------------------- python/pyspark/shell.py | 4 +- python/pyspark/shell.pyi | 31 ------ python/pyspark/shuffle.pyi | 109 ------------------- python/pyspark/sql/avro/__init__.pyi | 22 ---- python/pyspark/sql/pandas/__init__.pyi | 17 --- python/pyspark/sql/pandas/serializers.pyi | 65 ------------ python/pyspark/sql/pandas/typehints.pyi | 33 ------ python/pyspark/sql/pandas/types.pyi | 41 -------- python/pyspark/sql/pandas/utils.pyi | 20 ---- python/pyspark/sql/utils.pyi | 55 ---------- python/pyspark/streaming/__init__.pyi | 23 ---- python/pyspark/streaming/util.pyi | 48 --------- python/pyspark/traceback_utils.pyi | 29 ----- python/pyspark/util.py | 2 +- python/pyspark/util.pyi | 35 ------- python/pyspark/worker.pyi | 73 ------------- 30 files changed, 14 insertions(+), 1039 deletions(-) delete mode 100644 python/pyspark/_globals.pyi delete mode 100644 python/pyspark/daemon.pyi delete mode 100644 python/pyspark/find_spark_home.pyi delete mode 100644 python/pyspark/java_gateway.pyi delete mode 100644 python/pyspark/join.pyi delete mode 100644 python/pyspark/ml/__init__.pyi delete mode 100644 python/pyspark/mllib/__init__.pyi delete mode 100644 python/pyspark/rddsampler.pyi delete mode 100644 python/pyspark/resource/__init__.pyi delete mode 100644 python/pyspark/serializers.pyi delete mode 100644 python/pyspark/shell.pyi delete mode 100644 python/pyspark/shuffle.pyi delete mode 100644 python/pyspark/sql/avro/__init__.pyi delete mode 100644 python/pyspark/sql/pandas/__init__.pyi delete mode 100644 python/pyspark/sql/pandas/serializers.pyi delete mode 100644 python/pyspark/sql/pandas/typehints.pyi delete mode 100644 python/pyspark/sql/pandas/types.pyi delete mode 100644 python/pyspark/sql/pandas/utils.pyi delete mode 100644 python/pyspark/sql/utils.pyi delete mode 100644 python/pyspark/streaming/__init__.pyi delete mode 100644 python/pyspark/streaming/util.pyi delete mode 100644 python/pyspark/traceback_utils.pyi delete mode 100644 python/pyspark/util.pyi delete mode 100644 python/pyspark/worker.pyi diff --git a/python/mypy.ini b/python/mypy.ini index a9523e622ca0d..4a5368a519097 100644 --- a/python/mypy.ini +++ b/python/mypy.ini @@ -32,5 +32,8 @@ ignore_missing_imports = True [mypy-pandas.*] ignore_missing_imports = True -[mypy-pyarrow] +[mypy-pyarrow.*] +ignore_missing_imports = True + +[mypy-psutil.*] ignore_missing_imports = True diff --git a/python/pyspark/_globals.pyi b/python/pyspark/_globals.pyi deleted file mode 100644 index 9453775621196..0000000000000 --- a/python/pyspark/_globals.pyi +++ /dev/null @@ -1,27 +0,0 @@ -# -# 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. - -# NOTE: This dynamically typed stub was automatically generated by stubgen. - -from typing import Any - -__ALL__: Any - -class _NoValueType: - def __new__(cls): ... - def __reduce__(self): ... diff --git a/python/pyspark/accumulators.pyi b/python/pyspark/accumulators.pyi index 94f8023d1102b..13a1792cd247d 100644 --- a/python/pyspark/accumulators.pyi +++ b/python/pyspark/accumulators.pyi @@ -16,7 +16,7 @@ # specific language governing permissions and limitations # under the License. -from typing import Callable, Generic, Tuple, Type, TypeVar +from typing import Callable, Dict, Generic, Tuple, Type, TypeVar import socketserver.BaseRequestHandler # type: ignore @@ -27,6 +27,8 @@ U = TypeVar("U", bound=SupportsIAdd) import socketserver as SocketServer +_accumulatorRegistry: Dict[int, Accumulator] + class Accumulator(Generic[T]): aid: int accum_param: AccumulatorParam[T] diff --git a/python/pyspark/broadcast.pyi b/python/pyspark/broadcast.pyi index c2ea3c6f7d8b4..4b019a509a003 100644 --- a/python/pyspark/broadcast.pyi +++ b/python/pyspark/broadcast.pyi @@ -17,10 +17,12 @@ # under the License. import threading -from typing import Any, Generic, Optional, TypeVar +from typing import Any, Dict, Generic, Optional, TypeVar T = TypeVar("T") +_broadcastRegistry: Dict[int, Broadcast] + class Broadcast(Generic[T]): def __init__( self, diff --git a/python/pyspark/daemon.pyi b/python/pyspark/daemon.pyi deleted file mode 100644 index dfacf30a9f8a7..0000000000000 --- a/python/pyspark/daemon.pyi +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. - -from pyspark.serializers import ( # noqa: F401 - UTF8Deserializer as UTF8Deserializer, - read_int as read_int, - write_int as write_int, - write_with_length as write_with_length, -) -from typing import Any - -def compute_real_exit_code(exit_code: Any): ... -def worker(sock: Any, authenticated: Any): ... -def manager() -> None: ... diff --git a/python/pyspark/find_spark_home.pyi b/python/pyspark/find_spark_home.pyi deleted file mode 100644 index 217e5db960782..0000000000000 --- a/python/pyspark/find_spark_home.pyi +++ /dev/null @@ -1,17 +0,0 @@ -# -# 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. diff --git a/python/pyspark/java_gateway.pyi b/python/pyspark/java_gateway.pyi deleted file mode 100644 index 5b45206dc045c..0000000000000 --- a/python/pyspark/java_gateway.pyi +++ /dev/null @@ -1,24 +0,0 @@ -# -# 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. - -from pyspark.serializers import UTF8Deserializer as UTF8Deserializer, read_int as read_int, write_with_length as write_with_length # type: ignore[attr-defined] -from typing import Any, Optional - -def launch_gateway(conf: Optional[Any] = ..., popen_kwargs: Optional[Any] = ...): ... -def local_connect_and_auth(port: Any, auth_secret: Any): ... -def ensure_callback_server_started(gw: Any) -> None: ... diff --git a/python/pyspark/join.pyi b/python/pyspark/join.pyi deleted file mode 100644 index e89e0fbbcda9b..0000000000000 --- a/python/pyspark/join.pyi +++ /dev/null @@ -1,50 +0,0 @@ -# -# 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. - -from typing import Hashable, Iterable, Optional, Tuple, TypeVar - -from pyspark.resultiterable import ResultIterable -import pyspark.rdd - -K = TypeVar("K", bound=Hashable) -V = TypeVar("V") -U = TypeVar("U") - -def python_join( - rdd: pyspark.rdd.RDD[Tuple[K, V]], - other: pyspark.rdd.RDD[Tuple[K, U]], - numPartitions: int, -) -> pyspark.rdd.RDD[Tuple[K, Tuple[V, U]]]: ... -def python_right_outer_join( - rdd: pyspark.rdd.RDD[Tuple[K, V]], - other: pyspark.rdd.RDD[Tuple[K, U]], - numPartitions: int, -) -> pyspark.rdd.RDD[Tuple[K, Tuple[V, Optional[U]]]]: ... -def python_left_outer_join( - rdd: pyspark.rdd.RDD[Tuple[K, V]], - other: pyspark.rdd.RDD[Tuple[K, U]], - numPartitions: int, -) -> pyspark.rdd.RDD[Tuple[K, Tuple[Optional[V], U]]]: ... -def python_full_outer_join( - rdd: pyspark.rdd.RDD[Tuple[K, V]], - other: pyspark.rdd.RDD[Tuple[K, U]], - numPartitions: int, -) -> pyspark.rdd.RDD[Tuple[K, Tuple[Optional[V], Optional[U]]]]: ... -def python_cogroup( - rdds: Iterable[pyspark.rdd.RDD[Tuple[K, V]]], numPartitions: int -) -> pyspark.rdd.RDD[Tuple[K, Tuple[ResultIterable[V], ...]]]: ... diff --git a/python/pyspark/ml/__init__.pyi b/python/pyspark/ml/__init__.pyi deleted file mode 100644 index 8e3b8a5daeb08..0000000000000 --- a/python/pyspark/ml/__init__.pyi +++ /dev/null @@ -1,45 +0,0 @@ -# -# 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. - -from pyspark.ml import ( # noqa: F401 - classification as classification, - clustering as clustering, - evaluation as evaluation, - feature as feature, - fpm as fpm, - image as image, - linalg as linalg, - param as param, - recommendation as recommendation, - regression as regression, - stat as stat, - tuning as tuning, - util as util, -) -from pyspark.ml.base import ( # noqa: F401 - Estimator as Estimator, - Model as Model, - PredictionModel as PredictionModel, - Predictor as Predictor, - Transformer as Transformer, - UnaryTransformer as UnaryTransformer, -) -from pyspark.ml.pipeline import ( # noqa: F401 - Pipeline as Pipeline, - PipelineModel as PipelineModel, -) diff --git a/python/pyspark/mllib/__init__.pyi b/python/pyspark/mllib/__init__.pyi deleted file mode 100644 index 83032c4580fc8..0000000000000 --- a/python/pyspark/mllib/__init__.pyi +++ /dev/null @@ -1,32 +0,0 @@ -# -# 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. - -# NOTE: This dynamically typed stub was automatically generated by stubgen. - -# Names in __all__ with no definition: -# classification -# clustering -# feature -# fpm -# linalg -# random -# recommendation -# regression -# stat -# tree -# util diff --git a/python/pyspark/rddsampler.pyi b/python/pyspark/rddsampler.pyi deleted file mode 100644 index 8fbf72d90025c..0000000000000 --- a/python/pyspark/rddsampler.pyi +++ /dev/null @@ -1,54 +0,0 @@ -# -# 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. - -from typing import Any, Dict, Iterator, Optional, Tuple, TypeVar - -T = TypeVar("T") -U = TypeVar("U") -K = TypeVar("K") -V = TypeVar("V") - -class RDDSamplerBase: - def __init__(self, withReplacement: bool, seed: Optional[int] = ...) -> None: ... - def initRandomGenerator(self, split: int) -> None: ... - def getUniformSample(self) -> float: ... - def getPoissonSample(self, mean: float) -> int: ... - def func(self, split: int, iterator: Iterator[Any]) -> Iterator[Any]: ... - -class RDDSampler(RDDSamplerBase): - def __init__( - self, withReplacement: bool, fraction: float, seed: Optional[int] = ... - ) -> None: ... - def func(self, split: int, iterator: Iterator[T]) -> Iterator[T]: ... - -class RDDRangeSampler(RDDSamplerBase): - def __init__( - self, lowerBound: T, upperBound: T, seed: Optional[Any] = ... - ) -> None: ... - def func(self, split: int, iterator: Iterator[T]) -> Iterator[T]: ... - -class RDDStratifiedSampler(RDDSamplerBase): - def __init__( - self, - withReplacement: bool, - fractions: Dict[K, float], - seed: Optional[int] = ..., - ) -> None: ... - def func( - self, split: int, iterator: Iterator[Tuple[K, V]] - ) -> Iterator[Tuple[K, V]]: ... diff --git a/python/pyspark/resource/__init__.pyi b/python/pyspark/resource/__init__.pyi deleted file mode 100644 index 87a9b53c268ac..0000000000000 --- a/python/pyspark/resource/__init__.pyi +++ /dev/null @@ -1,31 +0,0 @@ -# -# 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. - -from pyspark.resource.information import ( # noqa: F401 - ResourceInformation as ResourceInformation, -) -from pyspark.resource.profile import ( # noqa: F401 - ResourceProfile as ResourceProfile, - ResourceProfileBuilder as ResourceProfileBuilder, -) -from pyspark.resource.requests import ( # noqa: F401 - ExecutorResourceRequest as ExecutorResourceRequest, - ExecutorResourceRequests as ExecutorResourceRequests, - TaskResourceRequest as TaskResourceRequest, - TaskResourceRequests as TaskResourceRequests, -) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 80ce9b8408d4e..e6033dd7505c1 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -342,7 +342,7 @@ def dumps(self, obj): # Hack namedtuple, make it picklable -__cls = {} +__cls = {} # type: ignore def _restore(name, fields, value): diff --git a/python/pyspark/serializers.pyi b/python/pyspark/serializers.pyi deleted file mode 100644 index 26ef17c38d227..0000000000000 --- a/python/pyspark/serializers.pyi +++ /dev/null @@ -1,122 +0,0 @@ -# -# 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. - -from typing import Any - -class SpecialLengths: - END_OF_DATA_SECTION: int = ... - PYTHON_EXCEPTION_THROWN: int = ... - TIMING_DATA: int = ... - END_OF_STREAM: int = ... - NULL: int = ... - START_ARROW_STREAM: int = ... - -class Serializer: - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - def load_stream(self, stream: Any) -> None: ... - def __eq__(self, other: Any) -> Any: ... - def __ne__(self, other: Any) -> Any: ... - def __hash__(self) -> Any: ... - -class FramedSerializer(Serializer): - def __init__(self) -> None: ... - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - def load_stream(self, stream: Any) -> None: ... - def dumps(self, obj: Any) -> None: ... - def loads(self, obj: Any) -> None: ... - -class BatchedSerializer(Serializer): - UNLIMITED_BATCH_SIZE: int = ... - UNKNOWN_BATCH_SIZE: int = ... - serializer: Any = ... - batchSize: Any = ... - def __init__(self, serializer: Any, batchSize: Any = ...) -> None: ... - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - def load_stream(self, stream: Any): ... - -class FlattenedValuesSerializer(BatchedSerializer): - def __init__(self, serializer: Any, batchSize: int = ...) -> None: ... - def load_stream(self, stream: Any): ... - -class AutoBatchedSerializer(BatchedSerializer): - bestSize: Any = ... - def __init__(self, serializer: Any, bestSize: Any = ...) -> None: ... - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - -class CartesianDeserializer(Serializer): - key_ser: Any = ... - val_ser: Any = ... - def __init__(self, key_ser: Any, val_ser: Any) -> None: ... - def load_stream(self, stream: Any): ... - -class PairDeserializer(Serializer): - key_ser: Any = ... - val_ser: Any = ... - def __init__(self, key_ser: Any, val_ser: Any) -> None: ... - def load_stream(self, stream: Any): ... - -class NoOpSerializer(FramedSerializer): - def loads(self, obj: Any): ... - def dumps(self, obj: Any): ... - -class PickleSerializer(FramedSerializer): - def dumps(self, obj: Any): ... - def loads(self, obj: Any, encoding: str = ...): ... - -class CloudPickleSerializer(PickleSerializer): - def dumps(self, obj: Any): ... - -class MarshalSerializer(FramedSerializer): - def dumps(self, obj: Any): ... - def loads(self, obj: Any): ... - -class AutoSerializer(FramedSerializer): - def __init__(self) -> None: ... - def dumps(self, obj: Any): ... - def loads(self, obj: Any): ... - -class CompressedSerializer(FramedSerializer): - serializer: Any = ... - def __init__(self, serializer: Any) -> None: ... - def dumps(self, obj: Any): ... - def loads(self, obj: Any): ... - -class UTF8Deserializer(Serializer): - use_unicode: Any = ... - def __init__(self, use_unicode: bool = ...) -> None: ... - def loads(self, stream: Any): ... - def load_stream(self, stream: Any) -> None: ... - -class ChunkedStream: - buffer_size: Any = ... - buffer: Any = ... - current_pos: int = ... - wrapped: Any = ... - def __init__(self, wrapped: Any, buffer_size: Any) -> None: ... - def write(self, bytes: Any) -> None: ... - def close(self) -> None: ... - @property - def closed(self): ... - -def write_with_length(obj: Any, stream: Any): ... -def pack_long(value): ... -def read_int(stream): ... -def read_long(stream): ... -def read_bool(stream): ... -def write_int(value, stream): ... -def write_long(value, stream): ... diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index cde163bd2d73d..0c6cc1302ff62 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -32,10 +32,10 @@ if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) -SparkContext._ensure_initialized() +SparkContext._ensure_initialized() # type: ignore try: - spark = SparkSession._create_shell_session() + spark = SparkSession._create_shell_session() # type: ignore except Exception: import sys import traceback diff --git a/python/pyspark/shell.pyi b/python/pyspark/shell.pyi deleted file mode 100644 index 0760309542f8d..0000000000000 --- a/python/pyspark/shell.pyi +++ /dev/null @@ -1,31 +0,0 @@ -# -# 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. - -from pyspark import SparkConf as SparkConf # noqa: F401 -from pyspark.context import SparkContext as SparkContext -from pyspark.sql import SQLContext as SQLContext, SparkSession as SparkSession -from typing import Any, Callable - -from pyspark.sql.dataframe import DataFrame - -spark: SparkSession -sc: SparkContext -sql: Callable[[str], DataFrame] -sqlContext: SQLContext -sqlCtx: SQLContext -code: Any diff --git a/python/pyspark/shuffle.pyi b/python/pyspark/shuffle.pyi deleted file mode 100644 index 10648c51dca8f..0000000000000 --- a/python/pyspark/shuffle.pyi +++ /dev/null @@ -1,109 +0,0 @@ -# -# 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. - -from pyspark.serializers import ( # noqa: F401 - AutoBatchedSerializer as AutoBatchedSerializer, - BatchedSerializer as BatchedSerializer, - CompressedSerializer as CompressedSerializer, - FlattenedValuesSerializer as FlattenedValuesSerializer, - PickleSerializer as PickleSerializer, -) -from pyspark.util import fail_on_stopiteration as fail_on_stopiteration # noqa: F401 -from typing import Any, Optional - -process: Any - -def get_used_memory(): ... - -MemoryBytesSpilled: int -DiskBytesSpilled: int - -class Aggregator: - createCombiner: Any = ... - mergeValue: Any = ... - mergeCombiners: Any = ... - def __init__( - self, createCombiner: Any, mergeValue: Any, mergeCombiners: Any - ) -> None: ... - -class SimpleAggregator(Aggregator): - def __init__(self, combiner: Any): ... - -class Merger: - agg: Any = ... - def __init__(self, aggregator: Any) -> None: ... - def mergeValues(self, iterator: Any) -> None: ... - def mergeCombiners(self, iterator: Any) -> None: ... - def items(self) -> None: ... - -class ExternalMerger(Merger): - MAX_TOTAL_PARTITIONS: int = ... - memory_limit: Any = ... - serializer: Any = ... - localdirs: Any = ... - partitions: Any = ... - batch: Any = ... - scale: Any = ... - data: Any = ... - pdata: Any = ... - spills: int = ... - def __init__( - self, - aggregator: Any, - memory_limit: int = ..., - serializer: Optional[Any] = ..., - localdirs: Optional[Any] = ..., - scale: int = ..., - partitions: int = ..., - batch: int = ..., - ) -> None: ... - def mergeValues(self, iterator: Any) -> None: ... - def mergeCombiners(self, iterator: Any, limit: Optional[Any] = ...) -> None: ... - def items(self): ... - -class ExternalSorter: - memory_limit: Any = ... - local_dirs: Any = ... - serializer: Any = ... - def __init__(self, memory_limit: Any, serializer: Optional[Any] = ...) -> None: ... - def sorted(self, iterator: Any, key: Optional[Any] = ..., reverse: bool = ...): ... - -class ExternalList: - LIMIT: int = ... - values: Any = ... - count: Any = ... - def __init__(self, values: Any) -> None: ... - def __iter__(self) -> Any: ... - def __len__(self): ... - def append(self, value: Any) -> None: ... - def __del__(self) -> None: ... - -class ExternalListOfList(ExternalList): - count: Any = ... - def __init__(self, values: Any) -> None: ... - def append(self, value: Any) -> None: ... - def __iter__(self) -> Any: ... - -class GroupByKey: - iterator: Any = ... - def __init__(self, iterator: Any) -> None: ... - def __iter__(self) -> Any: ... - -class ExternalGroupBy(ExternalMerger): - SORT_KEY_LIMIT: int = ... - def flattened_serializer(self): ... diff --git a/python/pyspark/sql/avro/__init__.pyi b/python/pyspark/sql/avro/__init__.pyi deleted file mode 100644 index 0d7871da4c100..0000000000000 --- a/python/pyspark/sql/avro/__init__.pyi +++ /dev/null @@ -1,22 +0,0 @@ -# -# 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. - -# NOTE: This dynamically typed stub was automatically generated by stubgen. - -# Names in __all__ with no definition: -# functions diff --git a/python/pyspark/sql/pandas/__init__.pyi b/python/pyspark/sql/pandas/__init__.pyi deleted file mode 100644 index 217e5db960782..0000000000000 --- a/python/pyspark/sql/pandas/__init__.pyi +++ /dev/null @@ -1,17 +0,0 @@ -# -# 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. diff --git a/python/pyspark/sql/pandas/serializers.pyi b/python/pyspark/sql/pandas/serializers.pyi deleted file mode 100644 index 8be3c0dcbc9ad..0000000000000 --- a/python/pyspark/sql/pandas/serializers.pyi +++ /dev/null @@ -1,65 +0,0 @@ -# -# 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. - -from pyspark.serializers import ( # noqa: F401 - Serializer as Serializer, - UTF8Deserializer as UTF8Deserializer, - read_int as read_int, - write_int as write_int, -) -from typing import Any - -class SpecialLengths: - END_OF_DATA_SECTION: int = ... - PYTHON_EXCEPTION_THROWN: int = ... - TIMING_DATA: int = ... - END_OF_STREAM: int = ... - NULL: int = ... - START_ARROW_STREAM: int = ... - -class ArrowCollectSerializer(Serializer): - serializer: Any = ... - def __init__(self) -> None: ... - def dump_stream(self, iterator: Any, stream: Any): ... - def load_stream(self, stream: Any) -> None: ... - -class ArrowStreamSerializer(Serializer): - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - def load_stream(self, stream: Any) -> None: ... - -class ArrowStreamPandasSerializer(ArrowStreamSerializer): - def __init__( - self, timezone: Any, safecheck: Any, assign_cols_by_name: Any - ) -> None: ... - def arrow_to_pandas(self, arrow_column: Any): ... - def dump_stream(self, iterator: Any, stream: Any) -> None: ... - def load_stream(self, stream: Any) -> None: ... - -class ArrowStreamPandasUDFSerializer(ArrowStreamPandasSerializer): - def __init__( - self, - timezone: Any, - safecheck: Any, - assign_cols_by_name: Any, - df_for_struct: bool = ..., - ) -> None: ... - def arrow_to_pandas(self, arrow_column: Any): ... - def dump_stream(self, iterator: Any, stream: Any): ... - -class CogroupUDFSerializer(ArrowStreamPandasUDFSerializer): - def load_stream(self, stream: Any) -> None: ... diff --git a/python/pyspark/sql/pandas/typehints.pyi b/python/pyspark/sql/pandas/typehints.pyi deleted file mode 100644 index eea9c86225332..0000000000000 --- a/python/pyspark/sql/pandas/typehints.pyi +++ /dev/null @@ -1,33 +0,0 @@ -# -# 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. - -from pyspark.sql.pandas.utils import ( # noqa: F401 - require_minimum_pandas_version as require_minimum_pandas_version, -) -from typing import Any, Optional - -def infer_eval_type(sig: Any): ... -def check_tuple_annotation( - annotation: Any, parameter_check_func: Optional[Any] = ... -): ... -def check_iterator_annotation( - annotation: Any, parameter_check_func: Optional[Any] = ... -): ... -def check_union_annotation( - annotation: Any, parameter_check_func: Optional[Any] = ... -): ... diff --git a/python/pyspark/sql/pandas/types.pyi b/python/pyspark/sql/pandas/types.pyi deleted file mode 100644 index 5ae29bd273180..0000000000000 --- a/python/pyspark/sql/pandas/types.pyi +++ /dev/null @@ -1,41 +0,0 @@ -# -# 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. - -from pyspark.sql.types import ( # noqa: F401 - ArrayType as ArrayType, - BinaryType as BinaryType, - BooleanType as BooleanType, - ByteType as ByteType, - DateType as DateType, - DecimalType as DecimalType, - DoubleType as DoubleType, - FloatType as FloatType, - IntegerType as IntegerType, - LongType as LongType, - ShortType as ShortType, - StringType as StringType, - StructField as StructField, - StructType as StructType, - TimestampType as TimestampType, -) -from typing import Any - -def to_arrow_type(dt: Any): ... -def to_arrow_schema(schema: Any): ... -def from_arrow_type(at: Any): ... -def from_arrow_schema(arrow_schema: Any): ... diff --git a/python/pyspark/sql/pandas/utils.pyi b/python/pyspark/sql/pandas/utils.pyi deleted file mode 100644 index e4d315b0ce205..0000000000000 --- a/python/pyspark/sql/pandas/utils.pyi +++ /dev/null @@ -1,20 +0,0 @@ -# -# 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. - -def require_minimum_pandas_version() -> None: ... -def require_minimum_pyarrow_version() -> None: ... diff --git a/python/pyspark/sql/utils.pyi b/python/pyspark/sql/utils.pyi deleted file mode 100644 index c11e4bed54e7f..0000000000000 --- a/python/pyspark/sql/utils.pyi +++ /dev/null @@ -1,55 +0,0 @@ -# -# 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. - -# NOTE: This dynamically typed stub was automatically generated by stubgen. - -from pyspark import SparkContext as SparkContext # noqa: F401 -from typing import Any, Optional - -class CapturedException(Exception): - desc: Any = ... - stackTrace: Any = ... - cause: Any = ... - def __init__( - self, desc: Any, stackTrace: Any, cause: Optional[Any] = ... - ) -> None: ... - -class AnalysisException(CapturedException): ... -class ParseException(CapturedException): ... -class IllegalArgumentException(CapturedException): ... -class StreamingQueryException(CapturedException): ... -class QueryExecutionException(CapturedException): ... -class PythonException(CapturedException): ... -class UnknownException(CapturedException): ... - -def convert_exception(e: Any): ... -def capture_sql_exception(f: Any): ... -def install_exception_handler() -> None: ... -def toJArray(gateway: Any, jtype: Any, arr: Any): ... -def require_test_compiled() -> None: ... - -class ForeachBatchFunction: - sql_ctx: Any = ... - func: Any = ... - def __init__(self, sql_ctx: Any, func: Any) -> None: ... - error: Any = ... - def call(self, jdf: Any, batch_id: Any) -> None: ... - class Java: - implements: Any = ... - -def to_str(value: Any): ... diff --git a/python/pyspark/streaming/__init__.pyi b/python/pyspark/streaming/__init__.pyi deleted file mode 100644 index 281c06e51cc60..0000000000000 --- a/python/pyspark/streaming/__init__.pyi +++ /dev/null @@ -1,23 +0,0 @@ -# -# 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. - -from pyspark.streaming.context import StreamingContext as StreamingContext # noqa: F401 -from pyspark.streaming.dstream import DStream as DStream # noqa: F401 -from pyspark.streaming.listener import ( # noqa: F401 - StreamingListener as StreamingListener, -) diff --git a/python/pyspark/streaming/util.pyi b/python/pyspark/streaming/util.pyi deleted file mode 100644 index d552eb15f4818..0000000000000 --- a/python/pyspark/streaming/util.pyi +++ /dev/null @@ -1,48 +0,0 @@ -# -# 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. - -# NOTE: This dynamically typed stub was automatically generated by stubgen. - -from typing import Any, Optional - -class TransformFunction: - ctx: Any - func: Any - deserializers: Any - rdd_wrap_func: Any - failure: Any - def __init__(self, ctx, func, *deserializers) -> None: ... - def rdd_wrapper(self, func): ... - def call(self, milliseconds, jrdds): ... - def getLastFailure(self): ... - class Java: - implements: Any - -class TransformFunctionSerializer: - ctx: Any - serializer: Any - gateway: Any - failure: Any - def __init__(self, ctx, serializer, gateway: Optional[Any] = ...) -> None: ... - def dumps(self, id): ... - def loads(self, data): ... - def getLastFailure(self): ... - class Java: - implements: Any - -def rddToFileName(prefix, suffix, timestamp): ... diff --git a/python/pyspark/traceback_utils.pyi b/python/pyspark/traceback_utils.pyi deleted file mode 100644 index 33b1b7dc3227f..0000000000000 --- a/python/pyspark/traceback_utils.pyi +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. - -from collections import namedtuple -from typing import Any - -CallSite = namedtuple("CallSite", "function file linenum") - -def first_spark_call(): ... - -class SCCallSiteSync: - def __init__(self, sc: Any) -> None: ... - def __enter__(self) -> None: ... - def __exit__(self, type: Any, value: Any, tb: Any) -> None: ... diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 86e5ab5a01585..d2ca484e8ace6 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -23,7 +23,7 @@ from py4j.clientserver import ClientServer -__all__ = [] +__all__ = [] # type: ignore def print_exec(stream): diff --git a/python/pyspark/util.pyi b/python/pyspark/util.pyi deleted file mode 100644 index 023b409831459..0000000000000 --- a/python/pyspark/util.pyi +++ /dev/null @@ -1,35 +0,0 @@ -# -# 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. -# - -from typing import Any, Tuple -from pyspark._typing import F - -import threading - -def print_exec(stream: Any) -> None: ... - -class VersionUtils: - @staticmethod - def majorMinorVersion(sparkVersion: str) -> Tuple[int, int]: ... - -def fail_on_stopiteration(f: F) -> F: ... - -class InheritableThread(threading.Thread): - def __init__(self, target: Any, *args: Any, **kwargs: Any): ... - def __del__(self) -> None: ... diff --git a/python/pyspark/worker.pyi b/python/pyspark/worker.pyi deleted file mode 100644 index cc264823cc867..0000000000000 --- a/python/pyspark/worker.pyi +++ /dev/null @@ -1,73 +0,0 @@ -# -# 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. - -from pyspark import shuffle as shuffle -from pyspark.broadcast import Broadcast as Broadcast -from pyspark.files import SparkFiles as SparkFiles -from pyspark.java_gateway import local_connect_and_auth as local_connect_and_auth -from pyspark.rdd import PythonEvalType as PythonEvalType -from pyspark.resource import ResourceInformation as ResourceInformation -from pyspark.serializers import ( - BatchedSerializer as BatchedSerializer, - PickleSerializer as PickleSerializer, - SpecialLengths as SpecialLengths, - UTF8Deserializer as UTF8Deserializer, - read_bool as read_bool, - read_int as read_int, - read_long as read_long, - write_int as write_int, - write_long as write_long, - write_with_length as write_with_length, -) -from pyspark.sql.pandas.serializers import ( - ArrowStreamPandasUDFSerializer as ArrowStreamPandasUDFSerializer, - CogroupUDFSerializer as CogroupUDFSerializer, -) -from pyspark.sql.pandas.types import to_arrow_type as to_arrow_type -from pyspark.sql.types import StructType as StructType -from pyspark.taskcontext import ( - BarrierTaskContext as BarrierTaskContext, - TaskContext as TaskContext, -) -from pyspark.util import fail_on_stopiteration as fail_on_stopiteration -from typing import Any - -has_resource_module: bool -pickleSer: Any -utf8_deserializer: Any - -def report_times(outfile: Any, boot: Any, init: Any, finish: Any) -> None: ... -def add_path(path: Any) -> None: ... -def read_command(serializer: Any, file: Any): ... -def chain(f: Any, g: Any): ... -def wrap_udf(f: Any, return_type: Any): ... -def wrap_scalar_pandas_udf(f: Any, return_type: Any): ... -def wrap_pandas_iter_udf(f: Any, return_type: Any): ... -def wrap_cogrouped_map_pandas_udf(f: Any, return_type: Any, argspec: Any): ... -def wrap_grouped_map_pandas_udf(f: Any, return_type: Any, argspec: Any): ... -def wrap_grouped_agg_pandas_udf(f: Any, return_type: Any): ... -def wrap_window_agg_pandas_udf( - f: Any, return_type: Any, runner_conf: Any, udf_index: Any -): ... -def wrap_unbounded_window_agg_pandas_udf(f: Any, return_type: Any): ... -def wrap_bounded_window_agg_pandas_udf(f: Any, return_type: Any): ... -def read_single_udf( - pickleSer: Any, infile: Any, eval_type: Any, runner_conf: Any, udf_index: Any -): ... -def read_udfs(pickleSer: Any, infile: Any, eval_type: Any): ... -def main(infile: Any, outfile: Any) -> None: ... From 94d648dff5f24b4dea3873fd8e6609b1a099d0a2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 7 Oct 2020 20:16:40 +0900 Subject: [PATCH 59/59] [SPARK-33036][SQL] Refactor RewriteCorrelatedScalarSubquery code to replace exprIds in a bottom-up manner ### What changes were proposed in this pull request? This PR intends to refactor code in `RewriteCorrelatedScalarSubquery` for replacing `ExprId`s in a bottom-up manner instead of doing in a top-down one. This PR comes from the talk with cloud-fan in https://github.com/apache/spark/pull/29585#discussion_r490371252. ### Why are the changes needed? To improve code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #29913 from maropu/RefactorRewriteCorrelatedScalarSubquery. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/optimizer/subquery.scala | 80 ++++++++++++------- 1 file changed, 51 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index a168dcd7a83f5..f184253ef0595 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -338,20 +338,15 @@ object PullupCorrelatedPredicates extends Rule[LogicalPlan] with PredicateHelper object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { /** * Extract all correlated scalar subqueries from an expression. The subqueries are collected using - * the given collector. To avoid the reuse of `exprId`s, this method generates new `exprId` - * for the subqueries and rewrite references in the given `expression`. - * This method returns extracted subqueries and the corresponding `exprId`s and these values - * will be used later in `constructLeftJoins` for building the child plan that - * returns subquery output with the `exprId`s. + * the given collector. The expression is rewritten and returned. */ private def extractCorrelatedScalarSubqueries[E <: Expression]( expression: E, - subqueries: ArrayBuffer[(ScalarSubquery, ExprId)]): E = { + subqueries: ArrayBuffer[ScalarSubquery]): E = { val newExpression = expression transform { case s: ScalarSubquery if s.children.nonEmpty => - val newExprId = NamedExpression.newExprId - subqueries += s -> newExprId - s.plan.output.head.withExprId(newExprId) + subqueries += s + s.plan.output.head } newExpression.asInstanceOf[E] } @@ -512,19 +507,23 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { /** * Construct a new child plan by left joining the given subqueries to a base plan. + * This method returns the child plan and an attribute mapping + * for the updated `ExprId`s of subqueries. If the non-empty mapping returned, + * this rule will rewrite subquery references in a parent plan based on it. */ private def constructLeftJoins( child: LogicalPlan, - subqueries: ArrayBuffer[(ScalarSubquery, ExprId)]): LogicalPlan = { - subqueries.foldLeft(child) { - case (currentChild, (ScalarSubquery(query, conditions, _), newExprId)) => + subqueries: ArrayBuffer[ScalarSubquery]): (LogicalPlan, AttributeMap[Attribute]) = { + val subqueryAttrMapping = ArrayBuffer[(Attribute, Attribute)]() + val newChild = subqueries.foldLeft(child) { + case (currentChild, ScalarSubquery(query, conditions, _)) => val origOutput = query.output.head val resultWithZeroTups = evalSubqueryOnZeroTups(query) if (resultWithZeroTups.isEmpty) { // CASE 1: Subquery guaranteed not to have the COUNT bug Project( - currentChild.output :+ Alias(origOutput, origOutput.name)(exprId = newExprId), + currentChild.output :+ origOutput, Join(currentChild, query, LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // Subquery might have the COUNT bug. Add appropriate corrections. @@ -544,12 +543,13 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { if (havingNode.isEmpty) { // CASE 2: Subquery with no HAVING clause + val subqueryResultExpr = + Alias(If(IsNull(alwaysTrueRef), + resultWithZeroTups.get, + aggValRef), origOutput.name)() + subqueryAttrMapping += ((origOutput, subqueryResultExpr.toAttribute)) Project( - currentChild.output :+ - Alias( - If(IsNull(alwaysTrueRef), - resultWithZeroTups.get, - aggValRef), origOutput.name)(exprId = newExprId), + currentChild.output :+ subqueryResultExpr, Join(currentChild, Project(query.output :+ alwaysTrueExpr, query), LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) @@ -576,7 +576,9 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { (IsNull(alwaysTrueRef), resultWithZeroTups.get), (Not(havingNode.get.condition), Literal.create(null, aggValRef.dataType))), aggValRef), - origOutput.name)(exprId = newExprId) + origOutput.name)() + + subqueryAttrMapping += ((origOutput, caseExpr.toAttribute)) Project( currentChild.output :+ caseExpr, @@ -587,6 +589,20 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { } } } + (newChild, AttributeMap(subqueryAttrMapping.toSeq)) + } + + private def updateAttrs[E <: Expression]( + exprs: Seq[E], + attrMap: AttributeMap[Attribute]): Seq[E] = { + if (attrMap.nonEmpty) { + val newExprs = exprs.map { _.transform { + case a: AttributeReference => attrMap.getOrElse(a, a) + }} + newExprs.asInstanceOf[Seq[E]] + } else { + exprs + } } /** @@ -595,36 +611,42 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { */ def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput { case a @ Aggregate(grouping, expressions, child) => - val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] - val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val rewriteExprs = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) if (subqueries.nonEmpty) { // We currently only allow correlated subqueries in an aggregate if they are part of the // grouping expressions. As a result we need to replace all the scalar subqueries in the // grouping expressions by their result. val newGrouping = grouping.map { e => - subqueries.find(_._1.semanticEquals(e)).map(_._1.plan.output.head).getOrElse(e) + subqueries.find(_.semanticEquals(e)).map(_.plan.output.head).getOrElse(e) } - val newAgg = Aggregate(newGrouping, newExpressions, constructLeftJoins(child, subqueries)) + val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) + val newExprs = updateAttrs(rewriteExprs, subqueryAttrMapping) + val newAgg = Aggregate(newGrouping, newExprs, newChild) val attrMapping = a.output.zip(newAgg.output) newAgg -> attrMapping } else { a -> Nil } case p @ Project(expressions, child) => - val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] - val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val rewriteExprs = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) if (subqueries.nonEmpty) { - val newProj = Project(newExpressions, constructLeftJoins(child, subqueries)) + val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) + val newExprs = updateAttrs(rewriteExprs, subqueryAttrMapping) + val newProj = Project(newExprs, newChild) val attrMapping = p.output.zip(newProj.output) newProj -> attrMapping } else { p -> Nil } case f @ Filter(condition, child) => - val subqueries = ArrayBuffer.empty[(ScalarSubquery, ExprId)] - val newCondition = extractCorrelatedScalarSubqueries(condition, subqueries) + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val rewriteCondition = extractCorrelatedScalarSubqueries(condition, subqueries) if (subqueries.nonEmpty) { - val newProj = Project(f.output, Filter(newCondition, constructLeftJoins(child, subqueries))) + val (newChild, subqueryAttrMapping) = constructLeftJoins(child, subqueries) + val newCondition = updateAttrs(Seq(rewriteCondition), subqueryAttrMapping).head + val newProj = Project(f.output, Filter(newCondition, newChild)) val attrMapping = f.output.zip(newProj.output) newProj -> attrMapping } else {