You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2017/08/15 13:44:13 UTC

[01/13] drill git commit: DRILL-5601: Rollup of external sort fixes an improvements

Repository: drill
Updated Branches:
  refs/heads/master 5c57b50f2 -> 073ea6819


http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
index e249c19..d2ff805 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.ops.OperExecContext;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.record.BatchSchema;
@@ -90,8 +91,15 @@ public class TestSortImpl extends DrillTest {
           .setQueryId(queryId)
           .build();
     SortConfig sortConfig = new SortConfig(opContext.getConfig());
+    DrillbitEndpoint ep = DrillbitEndpoint.newBuilder()
+        .setAddress("foo.bar.com")
+        .setUserPort(1234)
+        .setControlPort(1235)
+        .setDataPort(1236)
+        .setVersion("1.11")
+        .build();
     SpillSet spillSet = new SpillSet(opContext.getConfig(), handle,
-                                     popConfig);
+                                     popConfig, ep);
     PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(opContext);
     SpilledRuns spilledRuns = new SpilledRuns(opContext, spillSet, copierHolder);
     return new SortImpl(opContext, sortConfig, spilledRuns, outputBatch);

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
index 4200073..f525020 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -119,6 +119,7 @@ public class TestDrillbitResilience extends DrillTest {
     }
 
     try {
+      @SuppressWarnings("resource")
       final Drillbit drillbit = Drillbit.start(zkHelper.getConfig(), remoteServiceSet);
       drillbits.put(name, drillbit);
     } catch (final DrillbitStartupException e) {
@@ -132,6 +133,7 @@ public class TestDrillbitResilience extends DrillTest {
    * @param name name of the drillbit
    */
   private static void stopDrillbit(final String name) {
+    @SuppressWarnings("resource")
     final Drillbit drillbit = drillbits.get(name);
     if (drillbit == null) {
       throw new IllegalStateException("No Drillbit named \"" + name + "\" found");
@@ -168,8 +170,8 @@ public class TestDrillbitResilience extends DrillTest {
    * @param name name of the drillbit
    * @return endpoint of the drillbit
    */
+  @SuppressWarnings("resource")
   private static DrillbitEndpoint getEndpoint(final String name) {
-    @SuppressWarnings("resource")
     final Drillbit drillbit = drillbits.get(name);
     if (drillbit == null) {
       throw new IllegalStateException("No Drillbit named \"" + name + "\" found.");
@@ -508,9 +510,11 @@ public class TestDrillbitResilience extends DrillTest {
   }
 
   /**
-   * Given the result of {@link WaitUntilCompleteListener#waitForCompletion}, this method fails if the completed state
-   * is not as expected, or if an exception is thrown. The completed state could be COMPLETED or CANCELED. This state
-   * is set when {@link WaitUntilCompleteListener#queryCompleted} is called.
+   * Given the result of {@link WaitUntilCompleteListener#waitForCompletion},
+   * this method fails if the completed state is not as expected, or if an
+   * exception is thrown. The completed state could be COMPLETED or CANCELED.
+   * This state is set when {@link WaitUntilCompleteListener#queryCompleted} is
+   * called.
    */
   private static void assertStateCompleted(final Pair<QueryState, Exception> result, final QueryState expectedState) {
     final QueryState actualState = result.getFirst();
@@ -758,8 +762,8 @@ public class TestDrillbitResilience extends DrillTest {
   }
 
   /**
-   * Test cancelling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
-   * Specifically tests cancelling fragment which has {@link MergingRecordBatch} blocked waiting for data.
+   * Test canceling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
+   * Specifically tests canceling fragment which has {@link MergingRecordBatch} blocked waiting for data.
    */
   @Test
   @Repeat(count = NUM_RUNS)
@@ -776,8 +780,8 @@ public class TestDrillbitResilience extends DrillTest {
   }
 
   /**
-   * Test cancelling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
-   * Specifically tests cancelling fragment which has {@link UnorderedReceiverBatch} blocked waiting for data.
+   * Test canceling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
+   * Specifically tests canceling fragment which has {@link UnorderedReceiverBatch} blocked waiting for data.
    */
   @Test
   @Repeat(count = NUM_RUNS)
@@ -931,7 +935,13 @@ public class TestDrillbitResilience extends DrillTest {
 
   @Test // DRILL-3065
   public void failsAfterMSorterSorting() {
-    final String query = "select n_name from cp.`tpch/nation.parquet` order by n_name";
+
+    // Note: must use an input table that returns more than one
+    // batch. The sort uses an optimization for single-batch inputs
+    // which bypasses the code where this partiucular fault is
+    // injected.
+
+    final String query = "select n_name from cp.`tpch/lineitem.parquet` order by n_name";
     final Class<? extends Exception> typeOfException = RuntimeException.class;
 
     final long before = countAllocatedMemory();
@@ -946,7 +956,13 @@ public class TestDrillbitResilience extends DrillTest {
 
   @Test // DRILL-3085
   public void failsAfterMSorterSetup() {
-    final String query = "select n_name from cp.`tpch/nation.parquet` order by n_name";
+
+    // Note: must use an input table that returns more than one
+    // batch. The sort uses an optimization for single-batch inputs
+    // which bypasses the code where this partiucular fault is
+    // injected.
+
+    final String query = "select n_name from cp.`tpch/lineitem.parquet` order by n_name";
     final Class<? extends Exception> typeOfException = RuntimeException.class;
 
     final long before = countAllocatedMemory();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
index 5a06ec2..4e44464 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
@@ -17,8 +17,13 @@
  */
 package org.apache.drill.test;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
 import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
 import java.util.List;
 import java.util.Properties;
 
@@ -231,4 +236,119 @@ public class ClientFixture implements AutoCloseable {
   public RowSetBuilder rowSetBuilder(BatchSchema schema) {
     return new RowSetBuilder(allocator(), schema);
   }
+
+  /**
+   * Very simple parser for semi-colon separated lists of SQL statements which
+   * handles quoted semicolons. Drill can execute only one statement at a time
+   * (without a trailing semi-colon.) This parser breaks up a statement list
+   * into single statements. Input:<code><pre>
+   * USE a.b;
+   * ALTER SESSION SET `foo` = ";";
+   * SELECT * FROM bar WHERE x = "\";";
+   * </pre><code>Output:
+   * <ul>
+   * <li><tt>USE a.b</tt></li>
+   * <li><tt>ALTER SESSION SET `foo` = ";"</tt></li>
+   * <li><tt>SELECT * FROM bar WHERE x = "\";"</tt></li>
+   */
+
+  public static class StatementParser {
+    private final Reader in;
+    private StringBuilder buf;
+
+    public StatementParser(Reader in) {
+      this.in = in;
+    }
+
+    public String parseNext() throws IOException {
+      boolean eof = false;
+      buf = new StringBuilder();
+      for (;;) {
+        int c = in.read();
+        if (c == -1) {
+          eof = true;
+          break;
+        }
+        if (c == ';') {
+          break;
+        }
+        buf.append((char) c);
+        if (c == '"' || c == '\'' || c == '`') {
+          int quote = c;
+          boolean escape = false;
+          for (;;) {
+            c = in.read();
+            if (c == -1) {
+              throw new IllegalArgumentException("Mismatched quote: " + (char) c);
+            }
+            buf.append((char) c);
+            if (! escape && c == quote) {
+              break;
+            }
+            escape = c == '\\';
+          }
+        }
+      }
+      String stmt = buf.toString().trim();
+      if (stmt.isEmpty() && eof) {
+        return null;
+      }
+      return stmt;
+    }
+  }
+
+  private boolean trace = false;
+
+  public void enableTrace(boolean flag) {
+    this.trace = flag;
+  }
+
+  public int exec(Reader in) throws IOException {
+    StatementParser parser = new StatementParser(in);
+    int count = 0;
+    for (;;) {
+      String stmt = parser.parseNext();
+      if (stmt == null) {
+        if (trace) {
+          System.out.println("----");
+        }
+        return count;
+      }
+      if (stmt.isEmpty()) {
+        continue;
+      }
+      if (trace) {
+        System.out.println("----");
+        System.out.println(stmt);
+      }
+      runSqlSilently(stmt);
+      count++;
+    }
+  }
+
+  /**
+   * Execute a set of statements from a file.
+   * @param stmts the set of statements, separated by semicolons
+   * @return the number of statements executed
+   */
+
+  public int exec(File source) throws FileNotFoundException, IOException {
+    try (Reader in = new BufferedReader(new FileReader(source))) {
+      return exec(in);
+    }
+  }
+
+  /**
+   * Execute a set of statements from a string.
+   * @param stmts the set of statements, separated by semicolons
+   * @return the number of statements executed
+   */
+
+  public int exec(String stmts) {
+    try (Reader in = new StringReader(stmts)) {
+      return exec(in);
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/test/ProfileParser.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ProfileParser.java b/exec/java-exec/src/test/java/org/apache/drill/test/ProfileParser.java
index 1dafef7..d9e344a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ProfileParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ProfileParser.java
@@ -545,7 +545,7 @@ public class ProfileParser {
 
       p = Pattern.compile("rowcount = ([\\d.E]+), cumulative cost = \\{([\\d.E]+) rows, ([\\d.E]+) cpu, ([\\d.E]+) io, ([\\d.E]+) network, ([\\d.E]+) memory\\}, id = (\\d+)");
       m = p.matcher(tail);
-      if (! m.matches()) {
+      if (! m.find()) {
         throw new IllegalStateException("Could not parse costs: " + tail);
       }
       estRows = Double.parseDouble(m.group(1));

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index f2a27c8..37fcdfd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -17,6 +17,11 @@
  */
 package org.apache.drill.test;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -48,6 +53,7 @@ import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
+import org.apache.drill.test.ClientFixture.StatementParser;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.RowSetReader;
@@ -230,6 +236,25 @@ public class QueryBuilder {
     return sql(String.format(query, args));
   }
 
+  /**
+   * Parse a single SQL statement (with optional ending semi-colon) from
+   * the file provided.
+   * @param file the file containing exactly one SQL statement, with
+   * optional ending semi-colon
+   * @return this builder
+   */
+
+  public QueryBuilder sql(File file) throws FileNotFoundException, IOException {
+    try (BufferedReader in = new BufferedReader(new FileReader(file))) {
+      StatementParser parser = new StatementParser(in);
+      String sql = parser.parseNext();
+      if (sql == null) {
+        throw new IllegalArgumentException("No query found");
+      }
+      return sql(sql);
+    }
+  }
+
   public QueryBuilder physical(String plan) {
     return query(QueryType.PHYSICAL, plan);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
index 5139086..a9feafd 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
@@ -81,7 +81,6 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     if (BaseAllocator.DEBUG) {
       historicalLog.recordEvent("create()");
     }
-
   }
 
   public DrillBuf reallocIfNeeded(final int size) {
@@ -184,15 +183,15 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   /**
    * Transfer the memory accounting ownership of this DrillBuf to another allocator. This will generate a new DrillBuf
    * that carries an association with the underlying memory of this DrillBuf. If this DrillBuf is connected to the
-   * owning BufferLedger of this memory, that memory ownership/accounting will be transferred to the taret allocator. If
+   * owning BufferLedger of this memory, that memory ownership/accounting will be transferred to the target allocator. If
    * this DrillBuf does not currently own the memory underlying it (and is only associated with it), this does not
    * transfer any ownership to the newly created DrillBuf.
-   *
+   * <p>
    * This operation has no impact on the reference count of this DrillBuf. The newly created DrillBuf with either have a
    * reference count of 1 (in the case that this is the first time this memory is being associated with the new
    * allocator) or the current value of the reference count for the other AllocationManager/BufferLedger combination in
    * the case that the provided allocator already had an association to this underlying memory.
-   *
+   * <p>
    * Transfers will always succeed, even if that puts the other allocator into an overlimit situation. This is possible
    * due to the fact that the original owning allocator may have allocated this memory out of a local reservation
    * whereas the target allocator may need to allocate new memory from a parent or RootAllocator. This operation is done
@@ -218,6 +217,13 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   }
 
   /**
+   * Visible only for memory allocation calculations.
+   *
+   * @return
+   */
+  public BufferLedger getLedger() { return ledger; }
+
+  /**
    * The outcome of a Transfer.
    */
   public class TransferResult {
@@ -236,7 +242,6 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       this.allocationFit = allocationFit;
       this.buffer = buffer;
     }
-
   }
 
   @Override
@@ -269,9 +274,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       throw new IllegalStateException(
           String.format("DrillBuf[%d] refCnt has gone negative. Buffer Info: %s", id, toVerboseString()));
     }
-
     return refCnt == 0;
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
index 34647f9..833a604 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.memory.BaseAllocator.Verbosity;
 import org.apache.drill.exec.metrics.DrillMetrics;
 import org.apache.drill.exec.ops.BufferManager;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -246,7 +247,6 @@ public class AllocationManager {
         owningLedger = target;
         return overlimit;
       }
-
     }
 
     /**
@@ -387,11 +387,10 @@ public class AllocationManager {
       }
 
       return buf;
-
     }
 
     /**
-     * What is the total size (in bytes) of memory underlying this ledger.
+     * The total size (in bytes) of memory underlying this ledger.
      *
      * @return Size in bytes
      */
@@ -400,7 +399,7 @@ public class AllocationManager {
     }
 
     /**
-     * How much memory is accounted for by this ledger. This is either getSize() if this is the owning ledger for the
+     * Amount of memory accounted for by this ledger. This is either getSize() if this is the owning ledger for the
      * memory or zero in the case that this is not the owning ledger associated with this memory.
      *
      * @return Amount of accounted(owned) memory associated with this ledger.
@@ -418,17 +417,17 @@ public class AllocationManager {
     /**
      * Package visible for debugging/verification only.
      */
-    UnsafeDirectLittleEndian getUnderlying() {
+    @VisibleForTesting
+    protected UnsafeDirectLittleEndian getUnderlying() {
       return underlying;
     }
 
     /**
      * Package visible for debugging/verification only.
      */
-    boolean isOwningLedger() {
+    @VisibleForTesting
+    protected boolean isOwningLedger() {
       return this == owningLedger;
     }
-
   }
-
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
index 8efa3ee..4e17eda 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
@@ -841,6 +841,7 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
 
   @Override
   public void write(DrillBuf buf, OutputStream out) throws IOException {
+    assert(buf.readerIndex() == 0);
     write(buf, buf.readableBytes(), out);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/codegen/includes/vv_imports.ftl
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/includes/vv_imports.ftl b/exec/vector/src/main/codegen/includes/vv_imports.ftl
index 87a2106..efca346 100644
--- a/exec/vector/src/main/codegen/includes/vv_imports.ftl
+++ b/exec/vector/src/main/codegen/includes/vv_imports.ftl
@@ -43,12 +43,14 @@ import org.apache.drill.exec.vector.complex.writer.*;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter;
 import org.apache.drill.exec.util.JsonStringArrayList;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
 
 import java.util.Arrays;
 import java.util.Random;
 import java.util.List;
+import java.util.Set;
 
 import java.io.Closeable;
 import java.io.InputStream;

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/codegen/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/FixedValueVectors.java b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
index 1e83a4f..5a53e21 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -266,8 +266,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   @Override
-  public int getPayloadByteCount() {
-    return getAccessor().getValueCount() * VALUE_WIDTH;
+  public int getPayloadByteCount(int valueCount) {
+    return valueCount * ${type.width};
   }
 
   private class TransferImpl implements TransferPair{

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index 0f8d90c..a2c0deb 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -15,12 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
 
 import java.lang.Override;
 import java.lang.UnsupportedOperationException;
+import java.util.Set;
 
 <@pp.dropOutputFile />
 <#list vv.types as type>
@@ -177,15 +179,16 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    return bits.getAllocatedByteCount() + values.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    bits.collectLedgers(ledgers);
+    values.collectLedgers(ledgers);
   }
 
   @Override
-  public int getPayloadByteCount() {
+  public int getPayloadByteCount(int valueCount) {
     // For nullable, we include all values, null or not, in computing
     // the value length.
-    return bits.getPayloadByteCount() + values.getPayloadByteCount();
+    return bits.getPayloadByteCount(valueCount) + values.getPayloadByteCount(valueCount);
   }
 
   <#if type.major == "VarLen">
@@ -225,7 +228,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   public void allocateNew(int valueCount) {
     try {
       values.allocateNew(valueCount);
-      bits.allocateNew(valueCount+1);
+      bits.allocateNew(valueCount);
     } catch(OutOfMemoryException e) {
       clear();
       throw e;

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/codegen/templates/UnionVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionVector.java b/exec/vector/src/main/codegen/templates/UnionVector.java
index 207e55a..2c732f4 100644
--- a/exec/vector/src/main/codegen/templates/UnionVector.java
+++ b/exec/vector/src/main/codegen/templates/UnionVector.java
@@ -29,9 +29,12 @@ package org.apache.drill.exec.vector.complex;
 
 <#include "/@includes/vv_imports.ftl" />
 import java.util.Iterator;
+import java.util.Set;
+
 import org.apache.drill.exec.vector.complex.impl.ComplexCopier;
 import org.apache.drill.exec.util.CallBack;
 import org.apache.drill.exec.expr.BasicTypeHelper;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -203,19 +206,18 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
-  public int getAllocatedByteCount() {
+  public void collectLedgers(Set<BufferLedger> ledgers) {
     // Most vectors are held inside the internal map.
 
-    int count = internalMap.getAllocatedByteCount();
+    internalMap.collectLedgers(ledgers);
     if (bit != null) {
-      count += bit.getAllocatedByteCount();
+      bit.collectLedgers(ledgers);
     }
-    return count;
   }
 
   @Override
-  public int getPayloadByteCount() {
-    return internalMap.getPayloadByteCount();
+  public int getPayloadByteCount(int valueCount) {
+    return internalMap.getPayloadByteCount(valueCount);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index 9a9e178..0eb8906 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -17,8 +17,10 @@
  */
 
 import java.lang.Override;
+import java.util.Set;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
@@ -247,27 +249,26 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    return offsetVector.getAllocatedByteCount() + super.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    offsetVector.collectLedgers(ledgers);
+    super.collectLedgers(ledgers);
   }
 
   @Override
-  public int getPayloadByteCount() {
-    UInt${type.width}Vector.Accessor a = offsetVector.getAccessor();
-    int count = a.getValueCount();
-    if (count == 0) {
+  public int getPayloadByteCount(int valueCount) {
+    if (valueCount == 0) {
       return 0;
-    } else {
-      // If 1 or more values, then the last value is set to
-      // the offset of the next value, which is the same as
-      // the length of existing values.
-      // In addition to the actual data bytes, we must also
-      // include the "overhead" bytes: the offset vector entries
-      // that accompany each column value. Thus, total payload
-      // size is consumed text bytes + consumed offset vector
-      // bytes.
-      return a.get(count-1) + offsetVector.getPayloadByteCount();
     }
+    // If 1 or more values, then the last value is set to
+    // the offset of the next value, which is the same as
+    // the length of existing values.
+    // In addition to the actual data bytes, we must also
+    // include the "overhead" bytes: the offset vector entries
+    // that accompany each column value. Thus, total payload
+    // size is consumed text bytes + consumed offset vector
+    // bytes.
+    return offsetVector.getAccessor().get(valueCount) +
+           offsetVector.getPayloadByteCount(valueCount);
   }
 
   private class TransferImpl implements TransferPair{
@@ -308,7 +309,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     if (size > MAX_ALLOCATION_SIZE) {
       throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
     }
-    allocationSizeInBytes = (int)size;
+    allocationSizeInBytes = (int) size;
     offsetVector.setInitialCapacity(valueCount + 1);
   }
 
@@ -385,7 +386,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       throw new OversizedAllocationException("Unable to expand the buffer. Max allowed buffer size is reached.");
     }
 
-    logger.trace("Reallocating VarChar, new size {}",newAllocationSize);
+    logger.trace("Reallocating VarChar, new size {}", newAllocationSize);
     final DrillBuf newBuf = allocator.buffer((int)newAllocationSize);
     newBuf.setBytes(0, data, 0, data.capacity());
     data.release();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
index 77d457b..1401373 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,41 +21,42 @@ import org.apache.drill.exec.vector.complex.RepeatedFixedWidthVectorLike;
 import org.apache.drill.exec.vector.complex.RepeatedVariableWidthVectorLike;
 
 public class AllocationHelper {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocationHelper.class);
 
-  public static void allocate(ValueVector v, int valueCount, int bytesPerValue) {
-    allocate(v, valueCount, bytesPerValue, 5);
+  public static void allocate(ValueVector vector, int valueCount, int bytesPerValue) {
+    allocate(vector, valueCount, bytesPerValue, 5);
   }
 
-  public static void allocatePrecomputedChildCount(ValueVector v, int valueCount, int bytesPerValue, int childValCount){
-    if(v instanceof FixedWidthVector) {
-      ((FixedWidthVector) v).allocateNew(valueCount);
-    } else if (v instanceof VariableWidthVector) {
-      ((VariableWidthVector) v).allocateNew(valueCount * bytesPerValue, valueCount);
-    } else if(v instanceof RepeatedFixedWidthVectorLike) {
-      ((RepeatedFixedWidthVectorLike) v).allocateNew(valueCount, childValCount);
-    } else if(v instanceof RepeatedVariableWidthVectorLike) {
-      ((RepeatedVariableWidthVectorLike) v).allocateNew(childValCount * bytesPerValue, valueCount, childValCount);
+  public static void allocatePrecomputedChildCount(ValueVector vector, int valueCount, int bytesPerValue, int childValCount) {
+    if (vector instanceof FixedWidthVector) {
+      ((FixedWidthVector) vector).allocateNew(valueCount);
+    } else if (vector instanceof VariableWidthVector) {
+      ((VariableWidthVector) vector).allocateNew(valueCount * bytesPerValue, valueCount);
+    } else if (vector instanceof RepeatedFixedWidthVectorLike) {
+      ((RepeatedFixedWidthVectorLike) vector).allocateNew(valueCount, childValCount);
+    } else if (vector instanceof RepeatedVariableWidthVectorLike && childValCount > 0 && bytesPerValue > 0) {
+      // Assertion thrown if byte count is zero in the full allocateNew,
+      // so use default version instead.
+      ((RepeatedVariableWidthVectorLike) vector).allocateNew(childValCount * bytesPerValue, valueCount, childValCount);
     } else {
-      v.allocateNew();
+      vector.allocateNew();
     }
   }
 
-  public static void allocate(ValueVector v, int valueCount, int bytesPerValue, int repeatedPerTop){
-    allocatePrecomputedChildCount(v, valueCount, bytesPerValue, repeatedPerTop * valueCount);
+  public static void allocate(ValueVector vector, int valueCount, int bytesPerValue, int repeatedPerTop){
+    allocatePrecomputedChildCount(vector, valueCount, bytesPerValue, repeatedPerTop * valueCount);
   }
 
   /**
    * Allocates the exact amount if v is fixed width, otherwise falls back to dynamic allocation
-   * @param v value vector we are trying to allocate
+   * @param vector value vector we are trying to allocate
    * @param valueCount  size we are trying to allocate
    * @throws org.apache.drill.exec.memory.OutOfMemoryException if it can't allocate the memory
    */
-  public static void allocateNew(ValueVector v, int valueCount) {
-    if (v instanceof  FixedWidthVector) {
-      ((FixedWidthVector) v).allocateNew(valueCount);
+  public static void allocateNew(ValueVector vector, int valueCount) {
+    if (vector instanceof  FixedWidthVector) {
+      ((FixedWidthVector) vector).allocateNew(valueCount);
     } else {
-      v.allocateNew();
+      vector.allocateNew();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index 5ce58ed..e98a417 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -18,7 +18,11 @@
 package org.apache.drill.exec.vector;
 
 import io.netty.buffer.DrillBuf;
+
+import java.util.Set;
+
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.record.MaterializedField;
 
 
@@ -87,9 +91,6 @@ public abstract class BaseDataValueVector extends BaseValueVector {
   public void reset() {}
 
   @Override
-  public int getAllocatedByteCount() { return data.capacity(); }
-
-  @Override
   public void exchange(ValueVector other) {
     BaseDataValueVector target = (BaseDataValueVector) other;
     DrillBuf temp = data;
@@ -99,4 +100,11 @@ public abstract class BaseDataValueVector extends BaseValueVector {
     getMutator().exchange(target.getMutator());
     // No state in an Accessor to reset
   }
+
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    BufferLedger ledger = data.getLedger();
+    if (ledger != null) {
+      ledgers.add(ledger);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 0062e77..a7c81de 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -528,8 +528,7 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   }
 
   @Override
-  public int getPayloadByteCount() {
-    // One byte per value
-    return valueCount;
+  public int getPayloadByteCount(int valueCount) {
+    return getSizeFromCount(valueCount);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
index bd8566d..3136e32 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
@@ -22,9 +22,11 @@ import io.netty.buffer.DrillBuf;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.drill.exec.expr.holders.ObjectHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
@@ -229,13 +231,10 @@ public class ObjectVector extends BaseValueVector {
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    // Values not stored in direct memory?
-    return 0;
-  }
+  public void collectLedgers(Set<BufferLedger> ledgers) {}
 
   @Override
-  public int getPayloadByteCount() {
+  public int getPayloadByteCount(int valueCount) {
     // Values not stored in direct memory?
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index 3bc43fa..2926862 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -18,10 +18,12 @@
 package org.apache.drill.exec.vector;
 
 import java.io.Closeable;
+import java.util.Set;
 
 import io.netty.buffer.DrillBuf;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
@@ -204,16 +206,19 @@ public interface ValueVector extends Closeable, Iterable<ValueVector> {
   void copyEntry(int toIndex, ValueVector from, int fromIndex);
 
   /**
-   * Return the total memory consumed by all buffers within this vector.
+   * Add the ledgers underlying the buffers underlying the components of the
+   * vector to the set provided. Used to determine actual memory allocation.
+   *
+   * @param ledgers set of ledgers to which to add ledgers for this vector
    */
 
-  int getAllocatedByteCount();
+  void collectLedgers(Set<BufferLedger> ledgers);
 
   /**
    * Return the number of value bytes consumed by actual data.
    */
 
-  int getPayloadByteCount();
+  int getPayloadByteCount(int valueCount);
 
   /**
    * Exchange state with another value vector of the same type.

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
index 9a0b6be..5786487 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
@@ -18,11 +18,13 @@
 package org.apache.drill.exec.vector;
 
 import java.util.Iterator;
+import java.util.Set;
 
 import com.google.common.collect.Iterators;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
@@ -158,11 +160,10 @@ public class ZeroVector implements ValueVector {
   public void copyEntry(int toIndex, ValueVector from, int fromIndex) { }
 
   @Override
-  public int getAllocatedByteCount() { return 0; }
+  public void exchange(ValueVector other) { }
 
-  @Override
-  public int getPayloadByteCount() { return 0; }
+  public void collectLedgers(Set<BufferLedger> ledgers) {}
 
   @Override
-  public void exchange(ValueVector other) { }
+  public int getPayloadByteCount(int valueCount) { return 0; }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
index baba086..30db41e 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
@@ -22,11 +22,13 @@ import io.netty.buffer.DrillBuf;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.drill.common.collections.MapWithOrdinal;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.util.CallBack;
 import org.apache.drill.exec.vector.ValueVector;
@@ -117,6 +119,7 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
    *
    * @return resultant {@link org.apache.drill.exec.vector.ValueVector}
    */
+  @SuppressWarnings("unchecked")
   @Override
   public <T extends ValueVector> T addOrGet(String name, TypeProtos.MajorType type, Class<T> clazz) {
     final ValueVector existing = getChild(name);
@@ -277,21 +280,18 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    int count = 0;
-
+  public void collectLedgers(Set<BufferLedger> ledgers) {
     for (final ValueVector v : vectors.values()) {
-      count += v.getAllocatedByteCount();
+      v.collectLedgers(ledgers);
     }
-    return count;
   }
 
   @Override
-  public int getPayloadByteCount() {
+  public int getPayloadByteCount(int valueCount) {
     int count = 0;
 
     for (final ValueVector v : vectors.values()) {
-      count += v.getPayloadByteCount();
+      count += v.getPayloadByteCount(valueCount);
     }
     return count;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
index 5b8f44d..2b41b8b 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
@@ -21,12 +21,14 @@ import io.netty.buffer.DrillBuf;
 
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.Set;
 
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.SchemaChangeRuntimeException;
 import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.AddOrGetResult;
@@ -182,6 +184,7 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
     return vector == DEFAULT_DATA_VECTOR ? 0:1;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(VectorDescriptor descriptor) {
     boolean created = false;
@@ -210,13 +213,15 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    return offsets.getAllocatedByteCount() + vector.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    offsets.collectLedgers(ledgers);
+    vector.collectLedgers(ledgers);
   }
 
   @Override
-  public int getPayloadByteCount() {
-    return offsets.getPayloadByteCount() + vector.getPayloadByteCount();
+  public int getPayloadByteCount(int valueCount) {
+    int entryCount = offsets.getAccessor().get(valueCount);
+    return offsets.getPayloadByteCount(valueCount) + vector.getPayloadByteCount(entryCount);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
index 7f0e939..c61fd00 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
@@ -22,6 +22,7 @@ import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
@@ -41,6 +42,7 @@ import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import org.apache.drill.exec.vector.complex.writer.FieldWriter;
 
 import java.util.List;
+import java.util.Set;
 
 public class ListVector extends BaseRepeatedValueVector {
 
@@ -323,12 +325,15 @@ public class ListVector extends BaseRepeatedValueVector {
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    return offsets.getAllocatedByteCount() + bits.getAllocatedByteCount() + super.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    offsets.collectLedgers(ledgers);
+    bits.collectLedgers(ledgers);
+    super.collectLedgers(ledgers);
   }
 
   @Override
-  public int getPayloadByteCount() {
-    return offsets.getPayloadByteCount() + bits.getPayloadByteCount() + super.getPayloadByteCount();
+  public int getPayloadByteCount(int valueCount) {
+    return offsets.getPayloadByteCount(valueCount) + bits.getPayloadByteCount(valueCount) +
+           super.getPayloadByteCount(valueCount);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
index 969c141..ab2c3d8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
@@ -23,6 +23,7 @@ import io.netty.buffer.DrillBuf;
 
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -30,6 +31,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.holders.ComplexHolder;
 import org.apache.drill.exec.expr.holders.RepeatedListHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
@@ -415,6 +417,7 @@ public class RepeatedListVector extends AbstractContainerVector
   public void allocateNew(int valueCount, int innerValueCount) {
     clear();
     getOffsetVector().allocateNew(valueCount + 1);
+    getOffsetVector().getMutator().setSafe(0, 0);
     getMutator().reset();
   }
 
@@ -435,14 +438,13 @@ public class RepeatedListVector extends AbstractContainerVector
     copyFromSafe(fromIndex, toIndex, (RepeatedListVector) from);
   }
 
-  @Override
-  public int getAllocatedByteCount() {
-    return delegate.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    delegate.collectLedgers(ledgers);
   }
 
   @Override
-  public int getPayloadByteCount() {
-    return delegate.getPayloadByteCount();
+  public int getPayloadByteCount(int valueCount) {
+    return delegate.getPayloadByteCount(valueCount);
   }
 
   @Override
@@ -450,5 +452,4 @@ public class RepeatedListVector extends AbstractContainerVector
     // TODO: Figure out how to test this scenario, then what to do...
     throw new UnsupportedOperationException("Exchange() not yet supported for repeated lists");
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
index 7ff36a7..be73fc8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
@@ -22,6 +22,7 @@ import io.netty.buffer.DrillBuf;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -33,6 +34,7 @@ import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.expr.holders.ComplexHolder;
 import org.apache.drill.exec.expr.holders.RepeatedMapHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
@@ -100,7 +102,7 @@ public class RepeatedMapVector extends AbstractMapVector
   public void allocateNew(int groupCount, int innerValueCount) {
     clear();
     try {
-      offsets.allocateNew(groupCount + 1);
+      allocateOffsetsNew(groupCount);
       for (ValueVector v : getChildren()) {
         AllocationHelper.allocatePrecomputedChildCount(v, groupCount, 50, innerValueCount);
       }
@@ -108,10 +110,14 @@ public class RepeatedMapVector extends AbstractMapVector
       clear();
       throw e;
     }
-    offsets.zeroVector();
     mutator.reset();
   }
 
+  public void allocateOffsetsNew(int groupCount) {
+    offsets.allocateNew(groupCount + 1);
+    offsets.zeroVector();
+  }
+
   public Iterator<String> fieldNameIterator() {
     return getChildFieldNames().iterator();
   }
@@ -128,11 +134,7 @@ public class RepeatedMapVector extends AbstractMapVector
     if (getAccessor().getValueCount() == 0) {
       return 0;
     }
-    long bufferSize = offsets.getBufferSize();
-    for (final ValueVector v : (Iterable<ValueVector>) this) {
-      bufferSize += v.getBufferSize();
-    }
-    return (int) bufferSize;
+    return offsets.getBufferSize() + super.getBufferSize();
   }
 
   @Override
@@ -141,7 +143,7 @@ public class RepeatedMapVector extends AbstractMapVector
       return 0;
     }
 
-    long bufferSize = 0;
+    long bufferSize = offsets.getBufferSizeFor(valueCount);
     for (final ValueVector v : (Iterable<ValueVector>) this) {
       bufferSize += v.getBufferSizeFor(valueCount);
     }
@@ -451,7 +453,7 @@ public class RepeatedMapVector extends AbstractMapVector
       bufOffset += vectorLength;
     }
 
-    assert bufOffset == buffer.capacity();
+    assert bufOffset == buffer.writerIndex();
   }
 
 
@@ -598,7 +600,8 @@ public class RepeatedMapVector extends AbstractMapVector
   }
 
   @Override
-  public int getAllocatedByteCount() {
-    return super.getAllocatedByteCount( ) + offsets.getAllocatedByteCount();
+  public void collectLedgers(Set<BufferLedger> ledgers) {
+    super.collectLedgers(ledgers);
+    offsets.collectLedgers(ledgers);
   }
 }


[05/13] drill git commit: DRILL-5714: Fix NPE when mapr-db plugin is used in table function

Posted by jn...@apache.org.
DRILL-5714: Fix NPE when mapr-db plugin is used in table function

close #902


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/fd7fba6c
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/fd7fba6c
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/fd7fba6c

Branch: refs/heads/master
Commit: fd7fba6c56fb19c6fbf3074ce16b7c97d1aef63f
Parents: 8b56423
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Aug 10 14:34:13 2017 +0300
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 .../store/mapr/db/MapRDBFormatPluginConfig.java | 43 +++++-------------
 .../store/mapr/db/MapRDBScanBatchCreator.java   | 15 ++++---
 .../drill/exec/store/mapr/db/MapRDBSubScan.java | 46 ++++++++------------
 .../mapr/db/binary/BinaryTableGroupScan.java    |  7 ++-
 .../store/mapr/db/json/JsonTableGroupScan.java  |  6 +--
 .../mapr/db/json/MaprDBJsonRecordReader.java    |  5 +--
 6 files changed, 47 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
index 8b89b78..50a67b4 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPluginConfig.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,10 +21,10 @@ import org.apache.drill.exec.store.mapr.TableFormatPluginConfig;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
-import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
-@JsonTypeName("maprdb")  @JsonInclude(Include.NON_DEFAULT)
+@JsonTypeName("maprdb")
+@JsonInclude(Include.NON_DEFAULT)
 public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
 
   public boolean allTextMode = false;
@@ -35,17 +35,22 @@ public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
 
   @Override
   public int hashCode() {
-    return 53;
+    int result = (allTextMode ? 1231 : 1237);
+    result = 31 * result + (enablePushdown ? 1231 : 1237);
+    result = 31 * result + (ignoreSchemaChange ? 1231 : 1237);
+    result = 31 * result + (readAllNumbersAsDouble ? 1231 : 1237);
+    result = 31 * result + (disableCountOptimization ? 1231 : 1237);
+    return result;
   }
 
   @Override
   protected boolean impEquals(Object obj) {
-    MapRDBFormatPluginConfig other = (MapRDBFormatPluginConfig)obj;
+    MapRDBFormatPluginConfig other = (MapRDBFormatPluginConfig) obj;
     if (readAllNumbersAsDouble != other.readAllNumbersAsDouble) {
       return false;
     } else if (allTextMode != other.allTextMode) {
       return false;
-    } else if (isIgnoreSchemaChange() != other.isIgnoreSchemaChange()) {
+    } else if (ignoreSchemaChange != other.ignoreSchemaChange) {
       return false;
     } else if (enablePushdown != other.enablePushdown) {
       return false;
@@ -63,40 +68,16 @@ public class MapRDBFormatPluginConfig extends TableFormatPluginConfig {
     return allTextMode;
   }
 
-  @JsonProperty("allTextMode")
-  public void setAllTextMode(boolean mode) {
-    allTextMode = mode;
-  }
-
-  @JsonProperty("disableCountOptimization")
-  public void setDisableCountOptimization(boolean mode) {
-    disableCountOptimization = mode;
-  }
-
-  public boolean shouldDisableCountOptimization() {
+  public boolean disableCountOptimization() {
     return disableCountOptimization;
   }
 
-  @JsonProperty("readAllNumbersAsDouble")
-  public void setReadAllNumbersAsDouble(boolean read) {
-    readAllNumbersAsDouble = read;
-  }
-
   public boolean isEnablePushdown() {
     return enablePushdown;
   }
 
-  @JsonProperty("enablePushdown")
-  public void setEnablePushdown(boolean enablePushdown) {
-    this.enablePushdown = enablePushdown;
-  }
-
   public boolean isIgnoreSchemaChange() {
     return ignoreSchemaChange;
   }
 
-  public void setIgnoreSchemaChange(boolean ignoreSchemaChange) {
-    this.ignoreSchemaChange = ignoreSchemaChange;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
index c989bb0..d4a3f06 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBScanBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -34,7 +34,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBScanBatchCreator.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBScanBatchCreator.class);
 
   @Override
   public ScanBatch getBatch(FragmentContext context, MapRDBSubScan subScan, List<RecordBatch> children) throws ExecutionSetupException {
@@ -43,13 +43,16 @@ public class MapRDBScanBatchCreator implements BatchCreator<MapRDBSubScan>{
     for(MapRDBSubScanSpec scanSpec : subScan.getRegionScanSpecList()){
       try {
         if (BinaryTableGroupScan.TABLE_BINARY.equals(subScan.getTableType())) {
-          readers.add(new HBaseRecordReader(subScan.getFormatPlugin().getConnection(),
-              getHBaseSubScanSpec(scanSpec), subScan.getColumns(), context));
+          readers.add(new HBaseRecordReader(
+              subScan.getFormatPlugin().getConnection(),
+              getHBaseSubScanSpec(scanSpec),
+              subScan.getColumns(),
+              context));
         } else {
           readers.add(new MaprDBJsonRecordReader(scanSpec, subScan.getFormatPluginConfig(), subScan.getColumns(), context));
         }
-      } catch (Exception e1) {
-        throw new ExecutionSetupException(e1);
+      } catch (Exception e) {
+        throw new ExecutionSetupException(e);
       }
     }
     return new ScanBatch(subScan, context, readers.iterator());

http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
index 794141c..98335f3 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.mapr.db;
 import java.util.Iterator;
 import java.util.List;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -28,11 +29,9 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
@@ -41,40 +40,32 @@ import com.google.common.collect.ImmutableSet;
 // Class containing information for reading a single HBase region
 @JsonTypeName("maprdb-sub-scan")
 public class MapRDBSubScan extends AbstractBase implements SubScan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBSubScan.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBSubScan.class);
 
-  @JsonProperty
-  public final StoragePluginConfig storageConfig;
-  @JsonIgnore
-  private final MapRDBFormatPluginConfig formatPluginConfig;
-  private final FileSystemPlugin storagePlugin;
+  private final MapRDBFormatPlugin formatPlugin;
   private final List<MapRDBSubScanSpec> regionScanSpecList;
   private final List<SchemaPath> columns;
   private final String tableType;
 
-  private final MapRDBFormatPlugin formatPlugin;
-
   @JsonCreator
-  public MapRDBSubScan(@JacksonInject StoragePluginRegistry registry,
+  public MapRDBSubScan(@JacksonInject StoragePluginRegistry engineRegistry,
                        @JsonProperty("userName") String userName,
                        @JsonProperty("formatPluginConfig") MapRDBFormatPluginConfig formatPluginConfig,
-                       @JsonProperty("storageConfig") StoragePluginConfig storage,
+                       @JsonProperty("storageConfig") StoragePluginConfig storageConfig,
                        @JsonProperty("regionScanSpecList") List<MapRDBSubScanSpec> regionScanSpecList,
                        @JsonProperty("columns") List<SchemaPath> columns,
                        @JsonProperty("tableType") String tableType) throws ExecutionSetupException {
-    this(userName, formatPluginConfig,
-        (FileSystemPlugin) registry.getPlugin(storage),
-        storage, regionScanSpecList, columns, tableType);
+    this(userName,
+        (MapRDBFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatPluginConfig),
+        regionScanSpecList,
+        columns,
+        tableType);
   }
 
-  public MapRDBSubScan(String userName, MapRDBFormatPluginConfig formatPluginConfig, FileSystemPlugin storagePlugin, StoragePluginConfig storageConfig,
+  public MapRDBSubScan(String userName, MapRDBFormatPlugin formatPlugin,
       List<MapRDBSubScanSpec> maprSubScanSpecs, List<SchemaPath> columns, String tableType) {
     super(userName);
-    this.storageConfig = storageConfig;
-    this.storagePlugin = storagePlugin;
-    this.formatPluginConfig = formatPluginConfig;
-    this.formatPlugin = (MapRDBFormatPlugin) storagePlugin.getFormatPlugin(formatPluginConfig);
-
+    this.formatPlugin = formatPlugin;
     this.regionScanSpecList = maprSubScanSpecs;
     this.columns = columns;
     this.tableType = tableType;
@@ -101,7 +92,7 @@ public class MapRDBSubScan extends AbstractBase implements SubScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
-    return new MapRDBSubScan(getUserName(), formatPluginConfig, storagePlugin, storageConfig, regionScanSpecList, columns, tableType);
+    return new MapRDBSubScan(getUserName(), formatPlugin, regionScanSpecList, columns, tableType);
   }
 
   @Override
@@ -118,13 +109,14 @@ public class MapRDBSubScan extends AbstractBase implements SubScan {
     return tableType;
   }
 
-  public MapRDBFormatPluginConfig getFormatPluginConfig() {
-    return formatPluginConfig;
-  }
-
   @JsonIgnore
   public MapRDBFormatPlugin getFormatPlugin() {
     return formatPlugin;
   }
 
+  @JsonIgnore
+  public MapRDBFormatPluginConfig getFormatPluginConfig() {
+    return (MapRDBFormatPluginConfig) formatPlugin.getConfig();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
index c298456..282b848 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -129,7 +129,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
         tableStats = new MapRDBTableStats(getHBaseConf(), hbaseScanSpec.getTableName());
       }
       boolean foundStartRegion = false;
-      regionsToScan = new TreeMap<TabletFragmentInfo, String>();
+      regionsToScan = new TreeMap<>();
       List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
       for (HRegionLocation regionLocation : regionLocations) {
         HRegionInfo regionInfo = regionLocation.getRegionInfo();
@@ -178,8 +178,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
     assert minorFragmentId < endpointFragmentMapping.size() : String.format(
         "Mappings length [%d] should be greater than minor fragment id [%d] but it isn't.", endpointFragmentMapping.size(),
         minorFragmentId);
-    return new MapRDBSubScan(getUserName(), formatPluginConfig, getStoragePlugin(), getStoragePlugin().getConfig(),
-        endpointFragmentMapping.get(minorFragmentId), columns, TABLE_BINARY);
+    return new MapRDBSubScan(getUserName(), formatPlugin, endpointFragmentMapping.get(minorFragmentId), columns, TABLE_BINARY);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index 06c4e7a..a1d7f9a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -37,7 +37,6 @@ import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
 import org.apache.drill.exec.store.mapr.db.MapRDBGroupScan;
 import org.apache.drill.exec.store.mapr.db.MapRDBSubScan;
-import org.apache.drill.exec.store.mapr.db.MapRDBTableStats;
 import org.apache.drill.exec.store.mapr.db.TabletFragmentInfo;
 import org.apache.hadoop.conf.Configuration;
 import org.codehaus.jackson.annotate.JsonCreator;
@@ -180,8 +179,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan {
     assert minorFragmentId < endpointFragmentMapping.size() : String.format(
         "Mappings length [%d] should be greater than minor fragment id [%d] but it isn't.", endpointFragmentMapping.size(),
         minorFragmentId);
-    return new MapRDBSubScan(getUserName(), formatPluginConfig, getStoragePlugin(), getStoragePlugin().getConfig(),
-        endpointFragmentMapping.get(minorFragmentId), columns, TABLE_JSON);
+    return new MapRDBSubScan(getUserName(), formatPlugin, endpointFragmentMapping.get(minorFragmentId), columns, TABLE_JSON);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/fd7fba6c/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 3105bec..5921249 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -38,7 +38,6 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
 import org.apache.drill.exec.store.mapr.db.MapRDBSubScanSpec;
@@ -112,7 +111,7 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
       condition = com.mapr.db.impl.ConditionImpl.parseFrom(ByteBufs.wrap(serializedFilter));
     }
 
-    disableCountOptimization = formatPluginConfig.shouldDisableCountOptimization();
+    disableCountOptimization = formatPluginConfig.disableCountOptimization();
     setColumns(projectedColumns);
     unionEnabled = context.getOptions().getOption(ExecConstants.ENABLE_UNION_TYPE);
     readNumbersAsDouble = formatPluginConfig.isReadAllNumbersAsDouble();


[06/13] drill git commit: DRILL-4735: ConvertCountToDirectScan rule enhancements

Posted by jn...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
new file mode 100644
index 0000000..090f32f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.exec.store.pojo;
+
+import io.netty.buffer.DrillBuf;
+
+import java.sql.Timestamp;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.Float8Vector;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+
+import com.google.common.base.Charsets;
+
+public class PojoWriters {
+
+  /**
+   * Creates matching writer to the given field type.
+   *
+   * @param type field type
+   * @param fieldName field name
+   * @param buffer drill buffer
+   * @return pojo writer
+   * @throws ExecutionSetupException in case if writer was not found for the given type
+   */
+  public static PojoWriter getWriter(Class<?> type, String fieldName, DrillBuf buffer) throws ExecutionSetupException {
+
+    if (type == Integer.class) {
+      return new NIntWriter(fieldName);
+    } else if (type == Long.class) {
+      return new NBigIntWriter(fieldName);
+    } else if (type == Boolean.class) {
+      return new NBooleanWriter(fieldName);
+    } else if (type == Double.class) {
+      return new NDoubleWriter(fieldName);
+    } else if (type.isEnum()) {
+      return new EnumWriter(fieldName, buffer);
+    } else if (type == String.class) {
+      return new StringWriter(fieldName, buffer);
+    } else if (type == Timestamp.class) {
+      return new NTimeStampWriter(fieldName);
+      // primitives
+    } else if (type == int.class) {
+      return new IntWriter(fieldName);
+    } else if (type == double.class) {
+      return new DoubleWriter(fieldName);
+    } else if (type == boolean.class) {
+      return new BitWriter(fieldName);
+    } else if (type == long.class) {
+      return new LongWriter(fieldName);
+    }
+
+    throw new ExecutionSetupException(String.format("PojoRecordReader doesn't yet support conversions from the type [%s].", type));
+  }
+
+  /**
+   * Pojo writer for int. Does not expect to write null value.
+   */
+  public static class IntWriter extends AbstractPojoWriter<IntVector> {
+
+    public IntWriter(String fieldName) {
+      super(fieldName, Types.required(MinorType.INT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      vector.getMutator().setSafe(outboundIndex, (int) value);
+    }
+  }
+
+  /**
+   * Pojo writer for boolean. Does not expect to write null value.
+   */
+  public static class BitWriter extends AbstractPojoWriter<BitVector> {
+
+    public BitWriter(String fieldName) {
+      super(fieldName, Types.required(MinorType.BIT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      vector.getMutator().setSafe(outboundIndex, (boolean) value ? 1 : 0);
+    }
+
+  }
+
+  /**
+   * Pojo writer for long. Does not expect to write null value.
+   */
+  public static class LongWriter extends AbstractPojoWriter<BigIntVector> {
+
+    public LongWriter(String fieldName) {
+      super(fieldName, Types.required(MinorType.BIGINT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      vector.getMutator().setSafe(outboundIndex, (long) value);
+    }
+
+  }
+
+  /**
+   * Pojo writer for double. Does not expect to write null value.
+   */
+  public static class DoubleWriter extends AbstractPojoWriter<Float8Vector> {
+
+    public DoubleWriter(String fieldName) {
+      super(fieldName, Types.required(MinorType.FLOAT8));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      vector.getMutator().setSafe(outboundIndex, (double) value);
+    }
+
+  }
+
+  /**
+   * Parent class for String and Enum writers. Writes data using nullable varchar holder.
+   */
+  private abstract static class AbstractStringWriter extends AbstractPojoWriter<NullableVarCharVector> {
+    private DrillBuf data;
+    private final NullableVarCharHolder holder = new NullableVarCharHolder();
+
+    public AbstractStringWriter(String fieldName, DrillBuf managedBuf) {
+      super(fieldName, Types.optional(MinorType.VARCHAR));
+      this.data = managedBuf;
+      ensureLength(100);
+    }
+
+    void ensureLength(int len) {
+      data = data.reallocIfNeeded(len);
+    }
+
+    public void writeString(String s, int outboundIndex) {
+      holder.isSet = 1;
+      byte[] bytes = s.getBytes(Charsets.UTF_8);
+      ensureLength(bytes.length);
+      data.clear();
+      data.writeBytes(bytes);
+      holder.buffer = data;
+      holder.start = 0;
+      holder.end = bytes.length;
+      vector.getMutator().setSafe(outboundIndex, holder);
+    }
+  }
+
+  /**
+   * Pojo writer for Enum. If null is encountered does not write it.
+   */
+  public static class EnumWriter extends AbstractStringWriter{
+    public EnumWriter(String fieldName, DrillBuf managedBuf) {
+      super(fieldName, managedBuf);
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value == null) {
+        return;
+      }
+      writeString(((Enum<?>) value).name(), outboundIndex);
+    }
+  }
+
+  /**
+   * Pojo writer for String. If null is encountered does not write it.
+   */
+  public static class StringWriter extends AbstractStringWriter {
+    public StringWriter(String fieldName, DrillBuf managedBuf) {
+      super(fieldName, managedBuf);
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        writeString((String) value, outboundIndex);
+      }
+    }
+  }
+
+  /**
+   * Pojo writer for Integer. If null is encountered does not write it.
+   */
+  public static class NIntWriter extends AbstractPojoWriter<NullableIntVector> {
+
+    public NIntWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.INT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, (Integer) value);
+      }
+    }
+
+  }
+
+  /**
+   * Pojo writer for Long. If null is encountered does not write it.
+   */
+  public static class NBigIntWriter extends AbstractPojoWriter<NullableBigIntVector> {
+
+    public NBigIntWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.BIGINT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, (Long) value);
+      }
+    }
+
+  }
+
+  /**
+   * Pojo writer for Boolean. If null is encountered does not write it.
+   */
+  public static class NBooleanWriter extends AbstractPojoWriter<NullableBitVector> {
+
+    public NBooleanWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.BIT));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, (Boolean) value ? 1 : 0);
+      }
+    }
+
+  }
+
+  /**
+   * Pojo writer for Double. If null is encountered does not write it.
+   */
+  public static class NDoubleWriter extends AbstractPojoWriter<NullableFloat8Vector> {
+
+    public NDoubleWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.FLOAT8));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, (Double) value);
+      }
+    }
+
+  }
+
+  /**
+   * Pojo writer for Timestamp. If null is encountered does not write it.
+   */
+  public static class NTimeStampWriter extends AbstractPojoWriter<NullableTimeStampVector> {
+
+    public NTimeStampWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.TIMESTAMP));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, ((Timestamp) value).getTime());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
deleted file mode 100644
index e52384e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
+++ /dev/null
@@ -1,274 +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.drill.exec.store.pojo;
-
-import io.netty.buffer.DrillBuf;
-
-import java.lang.reflect.Field;
-import java.sql.Timestamp;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
-import org.apache.drill.exec.vector.BigIntVector;
-import org.apache.drill.exec.vector.BitVector;
-import org.apache.drill.exec.vector.Float8Vector;
-import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.NullableBigIntVector;
-import org.apache.drill.exec.vector.NullableBitVector;
-import org.apache.drill.exec.vector.NullableFloat8Vector;
-import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableTimeStampVector;
-import org.apache.drill.exec.vector.NullableVarCharVector;
-
-import com.google.common.base.Charsets;
-
-public class Writers {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Writers.class);
-
-  public static class IntWriter extends AbstractWriter<IntVector> {
-
-    public IntWriter(Field field) {
-      super(field, Types.required(MinorType.INT));
-      if (field.getType() != int.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      int i = field.getInt(pojo);
-      vector.getMutator().setSafe(outboundIndex, i);
-    }
-
-  }
-
-  public static class BitWriter extends AbstractWriter<BitVector>{
-
-    public BitWriter(Field field) {
-      super(field, Types.required(MinorType.BIT));
-      if (field.getType() != boolean.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      boolean b = field.getBoolean(pojo);
-      vector.getMutator().setSafe(outboundIndex, b ? 1 : 0);
-    }
-
-  }
-
-  public static class LongWriter extends AbstractWriter<BigIntVector>{
-
-    public LongWriter(Field field) {
-      super(field, Types.required(MinorType.BIGINT));
-      if (field.getType() != long.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      long l = field.getLong(pojo);
-      vector.getMutator().setSafe(outboundIndex, l);
-    }
-
-  }
-
-  public static class DoubleWriter extends AbstractWriter<Float8Vector>{
-
-    public DoubleWriter(Field field) {
-      super(field, Types.required(MinorType.FLOAT8));
-      if (field.getType() != double.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      double d = field.getDouble(pojo);
-
-      vector.getMutator().setSafe(outboundIndex, d);
-    }
-
-  }
-
-  private abstract static class AbstractStringWriter extends AbstractWriter<NullableVarCharVector>{
-    private DrillBuf data;
-    private final NullableVarCharHolder h = new NullableVarCharHolder();
-
-    public AbstractStringWriter(Field field, DrillBuf managedBuf) {
-      super(field, Types.optional(MinorType.VARCHAR));
-      this.data = managedBuf;
-      ensureLength(100);
-    }
-
-    void ensureLength(int len) {
-      data = data.reallocIfNeeded(len);
-    }
-
-    @Override
-    public void cleanup() {
-    }
-
-    public void writeString(String s, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      if (s == null) {
-        return;
-      } else {
-        h.isSet = 1;
-        byte[] bytes = s.getBytes(Charsets.UTF_8);
-        ensureLength(bytes.length);
-        data.clear();
-        data.writeBytes(bytes);
-        h.buffer = data;
-        h.start = 0;
-        h.end = bytes.length;
-        vector.getMutator().setSafe(outboundIndex, h);
-      }
-    }
-
-  }
-
-  public static class EnumWriter extends AbstractStringWriter{
-    public EnumWriter(Field field, DrillBuf managedBuf) {
-      super(field, managedBuf);
-      if (!field.getType().isEnum()) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Enum<?> e= ((Enum<?>) field.get(pojo));
-      if (e == null) {
-        return;
-      }
-      writeString(e.name(), outboundIndex);
-    }
-  }
-
-  public static class StringWriter extends AbstractStringWriter {
-    public StringWriter(Field field, DrillBuf managedBuf) {
-      super(field, managedBuf);
-      if (field.getType() != String.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      String s = (String) field.get(pojo);
-      writeString(s, outboundIndex);
-    }
-  }
-
-  public static class NIntWriter extends AbstractWriter<NullableIntVector>{
-
-    public NIntWriter(Field field) {
-      super(field, Types.optional(MinorType.INT));
-      if (field.getType() != Integer.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Integer i = (Integer) field.get(pojo);
-      if (i != null) {
-        vector.getMutator().setSafe(outboundIndex, i);
-      }
-    }
-
-  }
-
-  public static class NBigIntWriter extends AbstractWriter<NullableBigIntVector>{
-
-    public NBigIntWriter(Field field) {
-      super(field, Types.optional(MinorType.BIGINT));
-      if (field.getType() != Long.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Long o = (Long) field.get(pojo);
-      if (o != null) {
-        vector.getMutator().setSafe(outboundIndex, o);
-      }
-    }
-
-  }
-
-  public static class NBooleanWriter extends AbstractWriter<NullableBitVector>{
-
-    public NBooleanWriter(Field field) {
-      super(field, Types.optional(MinorType.BIT));
-      if (field.getType() != Boolean.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Boolean o = (Boolean) field.get(pojo);
-      if (o != null) {
-        vector.getMutator().setSafe(outboundIndex, o ? 1 : 0);
-      }
-    }
-
-  }
-  public static class NDoubleWriter extends AbstractWriter<NullableFloat8Vector>{
-
-    public NDoubleWriter(Field field) {
-      super(field, Types.optional(MinorType.FLOAT8));
-      if (field.getType() != Double.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Double o = (Double) field.get(pojo);
-      if (o != null) {
-        vector.getMutator().setSafe(outboundIndex, o);
-      }
-    }
-
-  }
-
-  public static class NTimeStampWriter extends AbstractWriter<NullableTimeStampVector>{
-
-    public NTimeStampWriter(Field field) {
-      super(field, Types.optional(MinorType.TIMESTAMP));
-      if (field.getType() != Timestamp.class) {
-        throw new IllegalStateException();
-      }
-    }
-
-    @Override
-    public void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
-      Timestamp o = (Timestamp) field.get(pojo);
-      if (o != null) {
-        vector.getMutator().setSafe(outboundIndex, o.getTime());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableBatchCreator.java
index 58bf433..2b0ef3f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.BatchCreator;
@@ -35,7 +36,6 @@ import org.apache.drill.exec.store.pojo.PojoRecordReader;
  * Local system tables do not require a full-fledged query because these records are present on every Drillbit.
  */
 public class SystemTableBatchCreator implements BatchCreator<SystemTableScan> {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemTableBatchCreator.class);
 
   @SuppressWarnings({ "rawtypes", "unchecked" })
   @Override
@@ -44,7 +44,7 @@ public class SystemTableBatchCreator implements BatchCreator<SystemTableScan> {
     throws ExecutionSetupException {
     final SystemTable table = scan.getTable();
     final Iterator<Object> iterator = table.getIterator(context);
-    final RecordReader reader = new PojoRecordReader(table.getPojoClass(), iterator);
+    final RecordReader reader = new PojoRecordReader(table.getPojoClass(), ImmutableList.copyOf(iterator));
 
     return new ScanBatch(scan, context, Collections.singleton(reader).iterator());
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
index 43b594b..46a4823 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -256,7 +256,7 @@ public class TestFunctionsWithTypeExpoQueries extends BaseTestQuery {
         "where concat(a, 'asdf') = 'asdf'", root);
 
     // Validate the plan
-    final String[] expectedPlan = {"Scan.*a.parquet.*numFiles=1"};
+    final String[] expectedPlan = {"Scan.*a.parquet.*numFiles = 1"};
     final String[] excludedPlan = {"Filter"};
     PlanTestBase.testPlanMatchingPatterns(query, expectedPlan, excludedPlan);
 
@@ -265,7 +265,7 @@ public class TestFunctionsWithTypeExpoQueries extends BaseTestQuery {
         .sqlQuery(query)
         .ordered()
         .baselineColumns("col")
-        .baselineValues(1l)
+        .baselineValues(1L)
         .build()
         .run();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestConvertCountToDirectScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestConvertCountToDirectScan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestConvertCountToDirectScan.java
index 21b4c79..04fe913 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestConvertCountToDirectScan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestConvertCountToDirectScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,15 +18,16 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.ExecConstants;
 import org.junit.Test;
 
 public class TestConvertCountToDirectScan extends PlanTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestConvertCountToDirectScan.class);
 
   @Test
   public void ensureCaseDoesntConvertToDirectScan() throws Exception {
     testPlanMatchingPatterns(
-        "select count(case when n_name = 'ALGERIA' and n_regionkey = 2 then n_nationkey else null end) as cnt from dfs.`${WORKING_PATH}/src/test/resources/directcount.parquet`",
+        "select count(case when n_name = 'ALGERIA' and n_regionkey = 2 then n_nationkey else null end) as cnt\n" +
+            "from dfs.`${WORKING_PATH}/src/test/resources/directcount.parquet`",
         new String[] { "CASE" },
         new String[]{});
   }
@@ -36,7 +37,7 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
     final String sql = "select count(*) as cnt from cp.`tpch/nation.parquet`";
     testPlanMatchingPatterns(
         sql,
-        new String[] { "PojoRecordReader" },
+        new String[] { "DynamicPojoRecordReader" },
         new String[]{});
 
     testBuilder()
@@ -45,7 +46,6 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
         .baselineColumns("cnt")
         .baselineValues(25L)
         .go();
-
   }
 
   @Test
@@ -53,7 +53,7 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
     final String sql = "select count(100) as cnt from cp.`tpch/nation.parquet`";
     testPlanMatchingPatterns(
         sql,
-        new String[] { "PojoRecordReader" },
+        new String[] { "DynamicPojoRecordReader" },
         new String[]{});
 
     testBuilder()
@@ -62,7 +62,6 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
         .baselineColumns("cnt")
         .baselineValues(25L)
         .go();
-
   }
 
   @Test
@@ -70,7 +69,39 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
     final String sql = "select count(1 + 2) as cnt from cp.`tpch/nation.parquet`";
     testPlanMatchingPatterns(
         sql,
-        new String[] { "PojoRecordReader" },
+        new String[] { "DynamicPojoRecordReader" },
+        new String[]{});
+
+    testBuilder()
+        .sqlQuery(sql)
+        .unOrdered()
+        .baselineColumns("cnt")
+        .baselineValues(25L)
+        .go();
+  }
+
+  @Test
+  public void ensureDoesNotConvertForDirectoryColumns() throws Exception {
+    final String sql = "select count(dir0) as cnt from cp.`tpch/nation.parquet`";
+    testPlanMatchingPatterns(
+        sql,
+        new String[] { "ParquetGroupScan" },
+        new String[]{});
+
+    testBuilder()
+        .sqlQuery(sql)
+        .unOrdered()
+        .baselineColumns("cnt")
+        .baselineValues(0L)
+        .go();
+  }
+
+  @Test
+  public void ensureConvertForImplicitColumns() throws Exception {
+    final String sql = "select count(fqn) as cnt from cp.`tpch/nation.parquet`";
+    testPlanMatchingPatterns(
+        sql,
+        new String[] { "DynamicPojoRecordReader" },
         new String[]{});
 
     testBuilder()
@@ -79,7 +110,42 @@ public class TestConvertCountToDirectScan extends PlanTestBase {
         .baselineColumns("cnt")
         .baselineValues(25L)
         .go();
+  }
+
+  @Test
+  public void ensureConvertForSeveralColumns() throws Exception {
+    test("use %s", TEMP_SCHEMA);
+    final String tableName = "parquet_table_counts";
+
+    try {
+      final String newFqnColumnName = "new_fqn";
+      test("alter session set `%s` = '%s'", ExecConstants.IMPLICIT_FQN_COLUMN_LABEL, newFqnColumnName);
+      test("create table %s as select * from cp.`parquet/alltypes_optional.parquet`", tableName);
+      test("refresh table metadata %s", tableName);
+
+      final String sql = String.format("select\n" +
+          "count(%s) as implicit_count,\n" +
+          "count(*) as star_count,\n" +
+          "count(col_int) as int_column_count,\n" +
+          "count(col_vrchr) as vrchr_column_count\n" +
+          "from %s", newFqnColumnName, tableName);
+
+      testPlanMatchingPatterns(
+          sql,
+          new String[] { "DynamicPojoRecordReader" },
+          new String[]{});
+
+      testBuilder()
+          .sqlQuery(sql)
+          .unOrdered()
+          .baselineColumns("implicit_count", "star_count", "int_column_count", "vrchr_column_count")
+          .baselineValues(6L, 6L, 2L, 3L)
+          .go();
 
+    } finally {
+      test("alter session reset `%s`", ExecConstants.IMPLICIT_FQN_COLUMN_LABEL);
+      test("drop table if exists %s", tableName);
+    }
   }
 
 }


[08/13] drill git commit: DRILL-5699: Drill Web UI Page Source Has Links To External Sites

Posted by jn...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/js/jquery.min.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/jquery.min.js b/exec/java-exec/src/main/resources/rest/static/js/jquery.min.js
deleted file mode 100644
index 73f33fb..0000000
--- a/exec/java-exec/src/main/resources/rest/static/js/jquery.min.js
+++ /dev/null
@@ -1,4 +0,0 @@
-/*! jQuery v1.11.0 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */
-!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k="".trim,l={},m="1.11.0",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return n.each(this,a,b)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(th
 is,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return n
 ull!=a&&a==a.window},isNumeric:function(a){return a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(l.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=s(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!
 1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:k&&!k.call("\ufeff\xa0")?function(a){return null==a?"":k.call(a)}:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=s(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=d.call(argum
 ents,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||n.guid++,e):void 0},now:function(){return+new Date},support:l}),n.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s="sizzle"+-new Date,t=a.document,u=0,v=0,w=eb(),x=eb(),y=eb(),z=function(a,b){return a===b&&(j=!0),0},A="undefined",B=1<<31,C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=D.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},J="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",K="[\\x20\\t\\r\\n\\f]",L="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",M=L.replace("w","w#"),N="
 \\["+K+"*("+L+")"+K+"*(?:([*^$|!~]?=)"+K+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+M+")|)|)"+K+"*\\]",O=":("+L+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+N.replace(3,8)+")*)|.*)\\)|)",P=new RegExp("^"+K+"+|((?:^|[^\\\\])(?:\\\\.)*)"+K+"+$","g"),Q=new RegExp("^"+K+"*,"+K+"*"),R=new RegExp("^"+K+"*([>+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(O),U=new RegExp("^"+M+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L.replace("w","w*")+")"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+
 )|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=/'|\\/g,ab=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),bb=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{G.apply(D=H.call(t.childNodes),t.childNodes),D[t.childNodes.length].nodeType}catch(cb){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function db(a,b,d,e){var f,g,h,i,j,m,p,q,u,v;if((b?b.ownerDocument||b:t)!==l&&k(b),b=b||l,d=d||[],!a||"string"!=typeof a)return d;if(1!==(i=b.nodeType)&&9!==i)return[];if(n&&!e){if(f=Z.exec(a))if(h=f[1]){if(9===i){if(g=b.getElementById(h),!g||!g.parentNode)return d;if(g.id===h)return d.push(g),d}else if(b.ownerDocument&&(g=b.ownerDocument.getElementById(h))&&r(b,g)&&g.id===h)return d.push(g),d}else{if(f[2])return G.apply(d,b.getElementsByTagName(a)),d;if((h=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElemen
 tsByClassName(h)),d}if(c.qsa&&(!o||!o.test(a))){if(q=p=s,u=b,v=9===i&&a,1===i&&"object"!==b.nodeName.toLowerCase()){m=ob(a),(p=b.getAttribute("id"))?q=p.replace(_,"\\$&"):b.setAttribute("id",q),q="[id='"+q+"'] ",j=m.length;while(j--)m[j]=q+pb(m[j]);u=$.test(a)&&mb(b.parentNode)||b,v=m.join(",")}if(v)try{return G.apply(d,u.querySelectorAll(v)),d}catch(w){}finally{p||b.removeAttribute("id")}}}return xb(a.replace(P,"$1"),b,d,e)}function eb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function fb(a){return a[s]=!0,a}function gb(a){var b=l.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function hb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function ib(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||B)-(~a.sourceIndex||B);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function jb(a){return functi
 on(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function kb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function lb(a){return fb(function(b){return b=+b,fb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function mb(a){return a&&typeof a.getElementsByTagName!==A&&a}c=db.support={},f=db.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},k=db.setDocument=function(a){var b,e=a?a.ownerDocument||a:t,g=e.defaultView;return e!==l&&9===e.nodeType&&e.documentElement?(l=e,m=e.documentElement,n=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){k()},!1):g.attachEvent&&g.attachEvent("onunload",function(){k()})),c.attributes=gb(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=gb(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").le
 ngth}),c.getElementsByClassName=Y.test(e.getElementsByClassName)&&gb(function(a){return a.innerHTML="<div class='a'></div><div class='a i'></div>",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=gb(function(a){return m.appendChild(a).id=s,!e.getElementsByName||!e.getElementsByName(s).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==A&&n){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){var c=typeof a.getAttributeNode!==A&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==A?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getE
 lementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==A&&n?b.getElementsByClassName(a):void 0},p=[],o=[],(c.qsa=Y.test(e.querySelectorAll))&&(gb(function(a){a.innerHTML="<select t=''><option selected=''></option></select>",a.querySelectorAll("[t^='']").length&&o.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||o.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll(":checked").length||o.push(":checked")}),gb(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&o.push("name"+K+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||o.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),o.push(",.*:")})),(c.matchesSelector=Y.test(q=m.webkitMatchesSelector||m.mozMatchesSelector||m.oMatchesSelector||m.msMatchesSelector))&&gb(function(a){c.disconnectedMatch=q.call(a,"div"),q.call(a,"[s!='']:x"),p.push("!=",O)}),o=o.length&&new RegExp(o.join("|
 ")),p=p.length&&new RegExp(p.join("|")),b=Y.test(m.compareDocumentPosition),r=b||Y.test(m.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},z=b?function(a,b){if(a===b)return j=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===t&&r(t,a)?-1:b===e||b.ownerDocument===t&&r(t,b)?1:i?I.call(i,a)-I.call(i,b):0:4&d?-1:1)}:function(a,b){if(a===b)return j=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],k=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:i?I.call(i,a)-I.call(i,b):0;if(f===g)return ib(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)k.unshift(c);while(h[d]==
 =k[d])d++;return d?ib(h[d],k[d]):h[d]===t?-1:k[d]===t?1:0},e):l},db.matches=function(a,b){return db(a,null,null,b)},db.matchesSelector=function(a,b){if((a.ownerDocument||a)!==l&&k(a),b=b.replace(S,"='$1']"),!(!c.matchesSelector||!n||p&&p.test(b)||o&&o.test(b)))try{var d=q.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return db(b,l,null,[a]).length>0},db.contains=function(a,b){return(a.ownerDocument||a)!==l&&k(a),r(a,b)},db.attr=function(a,b){(a.ownerDocument||a)!==l&&k(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!n):void 0;return void 0!==f?f:c.attributes||!n?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},db.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},db.uniqueSort=function(a){var b,d=[],e=0,f=0;if(j=!c.detectDuplicates,i=!c.sortStable&&a.slice(0),a.sort(z),j){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return i=n
 ull,a},e=db.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=db.selectors={cacheLength:50,createPseudo:fb,match:V,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ab,bb),a[3]=(a[4]||a[5]||"").replace(ab,bb),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||db.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&db.error(a[0]),a},PSEUDO:function(a){var b,c=!a[5]&&a[2];return V.CHILD.test(a[0])?null:(a[3]&&void 0!==a[4]?a[2]=a[4]:c&&T.test(c)&&(b=ob(c,!0))&&(b=c.indexOf(")",c.length-b)-c.leng
 th)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ab,bb).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=w[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&w(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==A&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=db.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLower
 Case(),t=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&t){k=q[s]||(q[s]={}),j=k[a]||[],n=j[0]===u&&j[1],m=j[0]===u&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[u,n,m];break}}else if(t&&(j=(b[s]||(b[s]={}))[a])&&j[0]===u)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(t&&((l[s]||(l[s]={}))[a]=[u,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||db.error("unsupported pseudo: "+a);return e[s]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?fb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:fb(function(a){var b=[],c=[],d=g(a.replace(P,"$1"));retur
 n d[s]?fb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:fb(function(a){return function(b){return db(a,b).length>0}}),contains:fb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:fb(function(a){return U.test(a||"")||db.error("unsupported lang: "+a),a=a.replace(ab,bb).toLowerCase(),function(b){var c;do if(c=n?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===m},focus:function(a){return a===l.activeElement&&(!l.hasFocus||l.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"=
 ==b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:lb(function(){return[0]}),last:lb(function(a,b){return[b-1]}),eq:lb(function(a,b,c){return[0>c?c+b:c]}),even:lb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:lb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:lb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:lb(function(a,b,c){for(var d=0>c?c+b:c;++d<b;)a.push(d);return a})}},d.pseudos.nt
 h=d.pseudos.eq;for(b in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})d.pseudos[b]=jb(b);for(b in{submit:!0,reset:!0})d.pseudos[b]=kb(b);function nb(){}nb.prototype=d.filters=d.pseudos,d.setFilters=new nb;function ob(a,b){var c,e,f,g,h,i,j,k=x[a+" "];if(k)return b?0:k.slice(0);h=a,i=[],j=d.preFilter;while(h){(!c||(e=Q.exec(h)))&&(e&&(h=h.slice(e[0].length)||h),i.push(f=[])),c=!1,(e=R.exec(h))&&(c=e.shift(),f.push({value:c,type:e[0].replace(P," ")}),h=h.slice(c.length));for(g in d.filter)!(e=V[g].exec(h))||j[g]&&!(e=j[g](e))||(c=e.shift(),f.push({value:c,type:g,matches:e}),h=h.slice(c.length));if(!c)break}return b?h.length:h?db.error(a):x(a,i).slice(0)}function pb(a){for(var b=0,c=a.length,d="";c>b;b++)d+=a[b].value;return d}function qb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=v++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[u,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1==
 =b.nodeType||e){if(i=b[s]||(b[s]={}),(h=i[d])&&h[0]===u&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function rb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function sb(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function tb(a,b,c,d,e,f){return d&&!d[s]&&(d=tb(d)),e&&!e[s]&&(e=tb(e,f)),fb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||wb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:sb(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=sb(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?I.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=sb(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ub(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],i=g||d.relative[" "],j=
 g?1:0,k=qb(function(a){return a===b},i,!0),l=qb(function(a){return I.call(b,a)>-1},i,!0),m=[function(a,c,d){return!g&&(d||c!==h)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>j;j++)if(c=d.relative[a[j].type])m=[qb(rb(m),c)];else{if(c=d.filter[a[j].type].apply(null,a[j].matches),c[s]){for(e=++j;f>e;e++)if(d.relative[a[e].type])break;return tb(j>1&&rb(m),j>1&&pb(a.slice(0,j-1).concat({value:" "===a[j-2].type?"*":""})).replace(P,"$1"),c,e>j&&ub(a.slice(j,e)),f>e&&ub(a=a.slice(e)),f>e&&pb(a))}m.push(c)}return rb(m)}function vb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,i,j,k){var m,n,o,p=0,q="0",r=f&&[],s=[],t=h,v=f||e&&d.find.TAG("*",k),w=u+=null==t?1:Math.random()||.1,x=v.length;for(k&&(h=g!==l&&g);q!==x&&null!=(m=v[q]);q++){if(e&&m){n=0;while(o=a[n++])if(o(m,g,i)){j.push(m);break}k&&(u=w)}c&&((m=!o&&m)&&p--,f&&r.push(m))}if(p+=q,c&&q!==p){n=0;while(o=b[n++])o(r,s,g,i);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=E.call(j));s=sb(s)}G.apply(j,s),k&&!f&&s.length>0&&p+b.length>1&&db.uniqueS
 ort(j)}return k&&(u=w,h=t),r};return c?fb(f):f}g=db.compile=function(a,b){var c,d=[],e=[],f=y[a+" "];if(!f){b||(b=ob(a)),c=b.length;while(c--)f=ub(b[c]),f[s]?d.push(f):e.push(f);f=y(a,vb(e,d))}return f};function wb(a,b,c){for(var d=0,e=b.length;e>d;d++)db(a,b[d],c);return c}function xb(a,b,e,f){var h,i,j,k,l,m=ob(a);if(!f&&1===m.length){if(i=m[0]=m[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&c.getById&&9===b.nodeType&&n&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(ab,bb),b)||[])[0],!b)return e;a=a.slice(i.shift().value.length)}h=V.needsContext.test(a)?0:i.length;while(h--){if(j=i[h],d.relative[k=j.type])break;if((l=d.find[k])&&(f=l(j.matches[0].replace(ab,bb),$.test(i[0].type)&&mb(b.parentNode)||b))){if(i.splice(h,1),a=f.length&&pb(i),!a)return G.apply(e,f),e;break}}}return g(a,m)(f,b,!n,e,$.test(a)&&mb(b.parentNode)||b),e}return c.sortStable=s.split("").sort(z).join("")===s,c.detectDuplicates=!!j,k(),c.sortDetached=gb(function(a){return 1&a.compareDocumentPosition
 (l.createElement("div"))}),gb(function(a){return a.innerHTML="<a href='#'></a>","#"===a.firstChild.getAttribute("href")})||hb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&gb(function(a){return a.innerHTML="<input/>",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||hb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),gb(function(a){return null==a.getAttribute("disabled")})||hb(J,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),db}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=n.expr.match.needsContext,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^.[^:#\[\.,]*$/;function x(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a
 ,function(a){return a===b!==c});if("string"==typeof b){if(w.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>=0!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(x(this,a||[],!1))},not:function(a){return this.pushStack(x(this,a||[],!0))},is:function(a){return!!x(this,"string"==typeof a&&u.test(a)?n(a):a||[],!1).length}});var y,z=a.document,A=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,B=n.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a
 ){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:A.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||y).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:z,!0)),v.test(c[1])&&n.isPlainObject(b))for(c in b)n.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=z.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return y.find(a);this.length=1,this[0]=d}return this.context=z,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof y.ready?y.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};B.prototype=n.fn,y=n(z);var C=/^(?:parents|prev(?:Until|All))/,D={children:!0,contents:!0,next:!0,prev:!0};n.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!n(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];ret
 urn d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=u.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.unique(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.unique(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function E(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&
 11!==b.nodeType?b:null},parents:function(a){return n.dir(a,"parentNode")},parentsUntil:function(a,b,c){return n.dir(a,"parentNode",c)},next:function(a){return E(a,"nextSibling")},prev:function(a){return E(a,"previousSibling")},nextAll:function(a){return n.dir(a,"nextSibling")},prevAll:function(a){return n.dir(a,"previousSibling")},nextUntil:function(a,b,c){return n.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return n.dir(a,"previousSibling",c)},siblings:function(a){return n.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return n.sibling(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(D[a]||(e=n.unique(e)),C.test(a)&&(e=e.reverse())),this.pushStack(e)}});var F=/\S+/g,G={};function H(a){var b=G[a]={};return n.each(a.match(F)||[]
 ,function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?G[a]||H(a):n.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){n.each(b,function(b,c){var d=n.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&n.each(arguments,function(a,c){var d;while((d=n.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?n.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c
 .slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return
  e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&n.isFunction(a.promise)?e:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){if(a===!0?!--n.readyWait:!n.isReady){if(!z.body)return setTimeout(n.ready);n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(z,[n]),n.fn.trigger&&n(z).trigger("ready").off("ready"))}}});function J(){z.addEventListener?(z.removeEventListener(
 "DOMContentLoaded",K,!1),a.removeEventListener("load",K,!1)):(z.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(z.addEventListener||"load"===event.type||"complete"===z.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===z.readyState)setTimeout(n.ready);else if(z.addEventListener)z.addEventListener("DOMContentLoaded",K,!1),a.addEventListener("load",K,!1);else{z.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&z.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!n.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}J(),n.ready()}}()}return I.promise(b)};var L="undefined",M;for(M in n(l))break;l.ownLast="0"!==M,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c=z.getElementsByTagName("body")[0];c&&(a=z.createElement("div"),a.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",b=z.createElement("div"),c.ap
 pendChild(a).appendChild(b),typeof b.style.zoom!==L&&(b.style.cssText="border:0;margin:0;width:1px;padding:1px;display:inline;zoom:1",(l.inlineBlockNeedsLayout=3===b.offsetWidth)&&(c.style.zoom=1)),c.removeChild(a),a=b=null)}),function(){var a=z.createElement("div");if(null==l.deleteExpando){l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}}a=null}(),n.acceptData=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(n.acceptData(a)){va
 r f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f
-}}function S(a,b,c){if(n.acceptData(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)d=g[c].name,0===d.indexOf("dat
 a-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d]));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.
 length<c?n.queue(this[0],a):void 0===b?this:this.each(function(){var c=n.queue(this,a,b);n._queueHooks(this,a),"fx"===a&&"inprogress"!==c[0]&&n.dequeue(this,a)})},dequeue:function(a){return this.each(function(){n.dequeue(this,a)})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,b){var c,d=1,e=n.Deferred(),f=this,g=this.length,h=function(){--d||e.resolveWith(f,[f])};"string"!=typeof a&&(b=a,a=void 0),a=a||"fx";while(g--)c=n._data(f[g],a+"queueHooks"),c&&c.empty&&(d++,c.empty.add(h));return h(),e.promise(b)}});var T=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,U=["Top","Right","Bottom","Left"],V=function(a,b){return a=b||a,"none"===n.css(a,"display")||!n.contains(a.ownerDocument,a)},W=n.access=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===n.type(c)){e=!0;for(h in c)n.access(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,n.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(n(a),c)})),b))for(;i>h;h++)b(a[h],
 c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},X=/^(?:checkbox|radio)$/i;!function(){var a=z.createDocumentFragment(),b=z.createElement("div"),c=z.createElement("input");if(b.setAttribute("className","t"),b.innerHTML="  <link/><table></table><a href='/a'>a</a>",l.leadingWhitespace=3===b.firstChild.nodeType,l.tbody=!b.getElementsByTagName("tbody").length,l.htmlSerialize=!!b.getElementsByTagName("link").length,l.html5Clone="<:nav></:nav>"!==z.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,a.appendChild(c),l.appendChecked=c.checked,b.innerHTML="<textarea>x</textarea>",l.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,a.appendChild(b),b.innerHTML="<input type='radio' checked='checked' name='t'/>",l.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){l.noCloneEvent=!1}),b.cloneNode(!0).click()),null==l.deleteExpando){l.deleteExpando=!0;try{delete b.test}
 catch(d){l.deleteExpando=!1}}a=b=c=null}(),function(){var b,c,d=z.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),l[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var Y=/^(?:input|select|textarea)$/i,Z=/^key/,$=/^(?:mouse|contextmenu)|click/,_=/^(?:focusinfocus|focusoutblur)$/,ab=/^([^.]*)(?:\.(.+)|)$/;function bb(){return!0}function cb(){return!1}function db(){try{return z.activeElement}catch(a){}}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof n===L||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(F)||[""],h=b.length;while(h--)f=ab.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.spec
 ial[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(F)||[""],j=b.length;while(j--)if(h=ab.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.rem
 ove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,m,o=[d||z],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||z,3!==d.nodeType&&8!==d.nodeType&&!_.test(p+n.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[n.expando]?b:new n.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:n.makeArray(c,[b]),k=n.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!n.isWindow(d)){for(i=k.delegateType||p,_.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocum
 ent||z)&&o.push(l.defaultView||l.parentWindow||a)}m=0;while((h=o[m++])&&!b.isPropagationStopped())b.type=m>1?i:k.bindType||p,f=(n._data(h,"events")||{})[b.type]&&n._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&n.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&n.acceptData(d)&&g&&d[p]&&!n.isWindow(d)){l=d[g],l&&(d[g]=null),n.event.triggered=p;try{d[p]()}catch(r){}n.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace
 ))&&(a.handleObj=e,a.data=e.data,c=((n.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?n(c,this).index(i)>=0:n.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h<b.length&&g.push({elem:this,handlers:b.slice(h)}),g},fix:function(a){if(a[n.expando])return a;var b,c,d,e=a.type,f=a,g=this.fixHooks[e];g||(this.fixHooks[e]=g=$.test(e)?this.mouseHooks:Z.test(e)?this.keyHooks:{}),d=g.props?this.props.concat(g.props):this.props,a=new n.Event(f),b=d.length;while(b--)c=d[b],a[c]=f[c];return a.target||(a.target=f.srcElem
 ent||z),3===a.target.nodeType&&(a.target=a.target.parentNode),a.metaKey=!!a.metaKey,g.filter?g.filter(a,f):a},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(a,b){return null==a.which&&(a.which=null!=b.charCode?b.charCode:b.keyCode),a}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(a,b){var c,d,e,f=b.button,g=b.fromElement;return null==a.pageX&&null!=b.clientX&&(d=a.target.ownerDocument||z,e=d.documentElement,c=d.body,a.pageX=b.clientX+(e&&e.scrollLeft||c&&c.scrollLeft||0)-(e&&e.clientLeft||c&&c.clientLeft||0),a.pageY=b.clientY+(e&&e.scrollTop||c&&c.scrollTop||0)-(e&&e.clientTop||c&&c.clientTop||0)),!a.relatedTarget&&g&&(a.relatedTarget=g===a.target?b.toElement:g),a.which||void 0===f||(a.which=1&f?1:2&f?3:4&f?2:0),a}}
 ,special:{load:{noBubble:!0},focus:{trigger:function(){if(this!==db()&&this.focus)try{return this.focus(),!1}catch(a){}},delegateType:"focusin"},blur:{trigger:function(){return this===db()&&this.blur?(this.blur(),!1):void 0},delegateType:"focusout"},click:{trigger:function(){return n.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):void 0},_default:function(a){return n.nodeName(a.target,"a")}},beforeunload:{postDispatch:function(a){void 0!==a.result&&(a.originalEvent.returnValue=a.result)}}},simulate:function(a,b,c,d){var e=n.extend(new n.Event,c,{type:a,isSimulated:!0,originalEvent:{}});d?n.event.trigger(e,null,b):n.event.dispatch.call(b,e),e.isDefaultPrevented()&&c.preventDefault()}},n.removeEvent=z.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){var d="on"+b;a.detachEvent&&(typeof a[d]===L&&(a[d]=null),a.detachEvent(d,c))},n.Event=function(a,b){return this instanceof n.Event?(a&&a.type?(this.ori
 ginalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||void 0===a.defaultPrevented&&(a.returnValue===!1||a.getPreventDefault&&a.getPreventDefault())?bb:cb):this.type=a,b&&n.extend(this,b),this.timeStamp=a&&a.timeStamp||n.now(),void(this[n.expando]=!0)):new n.Event(a,b)},n.Event.prototype={isDefaultPrevented:cb,isPropagationStopped:cb,isImmediatePropagationStopped:cb,preventDefault:function(){var a=this.originalEvent;this.isDefaultPrevented=bb,a&&(a.preventDefault?a.preventDefault():a.returnValue=!1)},stopPropagation:function(){var a=this.originalEvent;this.isPropagationStopped=bb,a&&(a.stopPropagation&&a.stopPropagation(),a.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=bb,this.stopPropagation()}},n.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(a,b){n.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj;return(!e||e!==d&&!n.contains(d,e))&&(a.type=f.orig
 Type,c=f.handler.apply(this,arguments),a.type=b),c}}}),l.submitBubbles||(n.event.special.submit={setup:function(){return n.nodeName(this,"form")?!1:void n.event.add(this,"click._submit keypress._submit",function(a){var b=a.target,c=n.nodeName(b,"input")||n.nodeName(b,"button")?b.form:void 0;c&&!n._data(c,"submitBubbles")&&(n.event.add(c,"submit._submit",function(a){a._submit_bubble=!0}),n._data(c,"submitBubbles",!0))})},postDispatch:function(a){a._submit_bubble&&(delete a._submit_bubble,this.parentNode&&!a.isTrigger&&n.event.simulate("submit",this.parentNode,a,!0))},teardown:function(){return n.nodeName(this,"form")?!1:void n.event.remove(this,"._submit")}}),l.changeBubbles||(n.event.special.change={setup:function(){return Y.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(n.event.add(this,"propertychange._change",function(a){"checked"===a.originalEvent.propertyName&&(this._just_changed=!0)}),n.event.add(this,"click._change",function(a){this._just_changed&&!a.isT
 rigger&&(this._just_changed=!1),n.event.simulate("change",this,a,!0)})),!1):void n.event.add(this,"beforeactivate._change",function(a){var b=a.target;Y.test(b.nodeName)&&!n._data(b,"changeBubbles")&&(n.event.add(b,"change._change",function(a){!this.parentNode||a.isSimulated||a.isTrigger||n.event.simulate("change",this.parentNode,a,!0)}),n._data(b,"changeBubbles",!0))})},handle:function(a){var b=a.target;return this!==b||a.isSimulated||a.isTrigger||"radio"!==b.type&&"checkbox"!==b.type?a.handleObj.handler.apply(this,arguments):void 0},teardown:function(){return n.event.remove(this,"._change"),!Y.test(this.nodeName)}}),l.focusinBubbles||n.each({focus:"focusin",blur:"focusout"},function(a,b){var c=function(a){n.event.simulate(b,a.target,n.event.fix(a),!0)};n.event.special[b]={setup:function(){var d=this.ownerDocument||this,e=n._data(d,b);e||d.addEventListener(a,c,!0),n._data(d,b,(e||0)+1)},teardown:function(){var d=this.ownerDocument||this,e=n._data(d,b)-1;e?n._data(d,b,e):(d.removeEve
 ntListener(a,c,!0),n._removeData(d,b))}}}),n.fn.extend({on:function(a,b,c,d,e){var f,g;if("object"==typeof a){"string"!=typeof b&&(c=c||b,b=void 0);for(f in a)this.on(f,b,c,a[f],e);return this}if(null==c&&null==d?(d=b,c=b=void 0):null==d&&("string"==typeof b?(d=c,c=void 0):(d=c,c=b,b=void 0)),d===!1)d=cb;else if(!d)return this;return 1===e&&(g=d,d=function(a){return n().off(a),g.apply(this,arguments)},d.guid=g.guid||(g.guid=n.guid++)),this.each(function(){n.event.add(this,a,d,c,b)})},one:function(a,b,c,d){return this.on(a,b,c,d,1)},off:function(a,b,c){var d,e;if(a&&a.preventDefault&&a.handleObj)return d=a.handleObj,n(a.delegateTarget).off(d.namespace?d.origType+"."+d.namespace:d.origType,d.selector,d.handler),this;if("object"==typeof a){for(e in a)this.off(e,b,a[e]);return this}return(b===!1||"function"==typeof b)&&(c=b,b=void 0),c===!1&&(c=cb),this.each(function(){n.event.remove(this,a,c,b)})},trigger:function(a,b){return this.each(function(){n.event.trigger(a,b,this)})},triggerHan
 dler:function(a,b){var c=this[0];return c?n.event.trigger(a,b,c,!0):void 0}});function eb(a){var b=fb.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}var fb="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",gb=/ jQuery\d+="(?:null|\d+)"/g,hb=new RegExp("<(?:"+fb+")[\\s/>]","i"),ib=/^\s+/,jb=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,kb=/<([\w:]+)/,lb=/<tbody/i,mb=/<|&#?\w+;/,nb=/<(?:script|style|link)/i,ob=/checked\s*(?:[^=]|=\s*.checked.)/i,pb=/^$|\/(?:java|ecma)script/i,qb=/^true\/(.*)/,rb=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,sb={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody>
 <colgroup>","</colgroup></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:l.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},tb=eb(z),ub=tb.appendChild(z.createElement("div"));sb.optgroup=sb.option,sb.tbody=sb.tfoot=sb.colgroup=sb.caption=sb.thead,sb.th=sb.td;function vb(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==L?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==L?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,vb(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function wb(a){X.test(a.type)&&(a.defaultChecked=a.checked)}function xb(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function yb(a){return a.type=(null!==n.find.attr(a,"type"))+"/"+a.type,a}function zb(a){var b=qb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type
 "),a}function Ab(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}function Bb(a,b){if(1===b.nodeType&&n.hasData(a)){var c,d,e,f=n._data(a),g=n._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)n.event.add(b,c,h[c][d])}g.data&&(g.data=n.extend({},g.data))}}function Cb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!l.noCloneEvent&&b[n.expando]){e=n._data(b);for(d in e.events)n.removeEvent(b,d,e.handle);b.removeAttribute(n.expando)}"script"===c&&b.text!==a.text?(yb(b).text=a.text,zb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),l.html5Clone&&a.innerHTML&&!n.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&X.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}n.extend({clone:function(a,b,c){var d,e,f
 ,g,h,i=n.contains(a.ownerDocument,a);if(l.html5Clone||n.isXMLDoc(a)||!hb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(ub.innerHTML=a.outerHTML,ub.removeChild(f=ub.firstChild)),!(l.noCloneEvent&&l.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(d=vb(f),h=vb(a),g=0;null!=(e=h[g]);++g)d[g]&&Cb(e,d[g]);if(b)if(c)for(h=h||vb(a),d=d||vb(f),g=0;null!=(e=h[g]);g++)Bb(e,d[g]);else Bb(a,f);return d=vb(f,"script"),d.length>0&&Ab(d,!i&&vb(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,k,m=a.length,o=eb(b),p=[],q=0;m>q;q++)if(f=a[q],f||0===f)if("object"===n.type(f))n.merge(p,f.nodeType?[f]:f);else if(mb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(kb.exec(f)||["",""])[1].toLowerCase(),k=sb[i]||sb._default,h.innerHTML=k[1]+f.replace(jb,"<$1></$2>")+k[2],e=k[0];while(e--)h=h.lastChild;if(!l.leadingWhitespace&&ib.test(f)&&p.push(b.createTextNode(ib.exec(f)[0])),!l.tbody){f="table"!==i||lb.test(f)?"<table>"!==k[1]||lb.test(f)?0:h:h.firstCh
 ild,e=f&&f.childNodes.length;while(e--)n.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}n.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),l.appendChecked||n.grep(vb(p,"input"),wb),q=0;while(f=p[q++])if((!d||-1===n.inArray(f,d))&&(g=n.contains(f.ownerDocument,f),h=vb(o.appendChild(f),"script"),g&&Ab(h),c)){e=0;while(f=h[e++])pb.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=n.expando,j=n.cache,k=l.deleteExpando,m=n.event.special;null!=(d=a[h]);h++)if((b||n.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)m[e]?n.event.remove(d,e):n.removeEvent(d,e,g.handle);j[f]&&(delete j[f],k?delete d[i]:typeof d.removeAttribute!==L?d.removeAttribute(i):d[i]=null,c.push(f))}}}),n.fn.extend({text:function(a){return W(this,function(a){return void 0===a?n.text(this):this.empty().append((this[0]&&this[0].ownerDo
 cument||z).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?n.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||n.cleanData(vb(c)),c.parentNode&&(b&&n.contains(c.ownerDocument,c)&&Ab(vb(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&n.cleanData(vb(a,!1));while(a.firstChild)a.removeChild(a.fir
 stChild);a.options&&n.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return W(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(gb,""):void 0;if(!("string"!=typeof a||nb.test(a)||!l.htmlSerialize&&hb.test(a)||!l.leadingWhitespace&&ib.test(a)||sb[(kb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(jb,"<$1></$2>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(vb(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,n.cleanData(vb(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,k=this.length,m=this,o=k-1,p=a[0],q=
 n.isFunction(p);if(q||k>1&&"string"==typeof p&&!l.checkClone&&ob.test(p))return this.each(function(c){var d=m.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(k&&(i=n.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=n.map(vb(i,"script"),yb),f=g.length;k>j;j++)d=i,j!==o&&(d=n.clone(d,!0,!0),f&&n.merge(g,vb(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,n.map(g,zb),j=0;f>j;j++)d=g[j],pb.test(d.type||"")&&!n._data(d,"globalEval")&&n.contains(h,d)&&(d.src?n._evalUrl&&n._evalUrl(d.src):n.globalEval((d.text||d.textContent||d.innerHTML||"").replace(rb,"")));i=c=null}return this}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=0,e=[],g=n(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),n(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Db,Eb={};function Fb(b,c){var d=n(c
 .createElement(b)).appendTo(c.body),e=a.getDefaultComputedStyle?a.getDefaultComputedStyle(d[0]).display:n.css(d[0],"display");return d.detach(),e}function Gb(a){var b=z,c=Eb[a];return c||(c=Fb(a,b),"none"!==c&&c||(Db=(Db||n("<iframe frameborder='0' width='0' height='0'/>")).appendTo(b.documentElement),b=(Db[0].contentWindow||Db[0].contentDocument).document,b.write(),b.close(),c=Fb(a,b),Db.detach()),Eb[a]=c),c}!function(){var a,b,c=z.createElement("div"),d="-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;display:block;padding:0;margin:0;border:0";c.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",a=c.getElementsByTagName("a")[0],a.style.cssText="float:left;opacity:.5",l.opacity=/^0.5/.test(a.style.opacity),l.cssFloat=!!a.style.cssFloat,c.style.backgroundClip="content-box",c.cloneNode(!0).style.backgroundClip="",l.clearCloneStyle="content-box"===c.style.backgroundClip,a=c=null,l.shrinkWrapBlocks=function(){var a,c,e,f;if(
 null==b){if(a=z.getElementsByTagName("body")[0],!a)return;f="border:0;width:0;height:0;position:absolute;top:0;left:-9999px",c=z.createElement("div"),e=z.createElement("div"),a.appendChild(c).appendChild(e),b=!1,typeof e.style.zoom!==L&&(e.style.cssText=d+";width:1px;padding:1px;zoom:1",e.innerHTML="<div></div>",e.firstChild.style.width="5px",b=3!==e.offsetWidth),a.removeChild(c),a=c=e=null}return b}}();var Hb=/^margin/,Ib=new RegExp("^("+T+")(?!px)[a-z%]+$","i"),Jb,Kb,Lb=/^(top|right|bottom|left)$/;a.getComputedStyle?(Jb=function(a){return a.ownerDocument.defaultView.getComputedStyle(a,null)},Kb=function(a,b,c){var d,e,f,g,h=a.style;return c=c||Jb(a),g=c?c.getPropertyValue(b)||c[b]:void 0,c&&(""!==g||n.contains(a.ownerDocument,a)||(g=n.style(a,b)),Ib.test(g)&&Hb.test(b)&&(d=h.width,e=h.minWidth,f=h.maxWidth,h.minWidth=h.maxWidth=h.width=g,g=c.width,h.width=d,h.minWidth=e,h.maxWidth=f)),void 0===g?g:g+""}):z.documentElement.currentStyle&&(Jb=function(a){return a.currentStyle},Kb=fun
 ction(a,b,c){var d,e,f,g,h=a.style;return c=c||Jb(a),g=c?c[b]:void 0,null==g&&h&&h[b]&&(g=h[b]),Ib.test(g)&&!Lb.test(b)&&(d=h.left,e=a.runtimeStyle,f=e&&e.left,f&&(e.left=a.currentStyle.left),h.left="fontSize"===b?"1em":g,g=h.pixelLeft+"px",h.left=d,f&&(e.left=f)),void 0===g?g:g+""||"auto"});function Mb(a,b){return{get:function(){var c=a();if(null!=c)return c?void delete this.get:(this.get=b).apply(this,arguments)}}}!function(){var b,c,d,e,f,g,h=z.createElement("div"),i="border:0;width:0;height:0;position:absolute;top:0;left:-9999px",j="-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;display:block;padding:0;margin:0;border:0";h.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",b=h.getElementsByTagName("a")[0],b.style.cssText="float:left;opacity:.5",l.opacity=/^0.5/.test(b.style.opacity),l.cssFloat=!!b.style.cssFloat,h.style.backgroundClip="content-box",h.cloneNode(!0).style.backgroundClip="",l.clearCloneStyle="content-bo
 x"===h.style.backgroundClip,b=h=null,n.extend(l,{reliableHiddenOffsets:function(){if(null!=c)return c;var a,b,d,e=z.createElement("div"),f=z.getElementsByTagName("body")[0];if(f)return e.setAttribute("className","t"),e.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",a=z.createElement("div"),a.style.cssText=i,f.appendChild(a).appendChild(e),e.innerHTML="<table><tr><td></td><td>t</td></tr></table>",b=e.getElementsByTagName("td"),b[0].style.cssText="padding:0;margin:0;border:0;display:none",d=0===b[0].offsetHeight,b[0].style.display="",b[1].style.display="none",c=d&&0===b[0].offsetHeight,f.removeChild(a),e=f=null,c},boxSizing:function(){return null==d&&k(),d},boxSizingReliable:function(){return null==e&&k(),e},pixelPosition:function(){return null==f&&k(),f},reliableMarginRight:function(){var b,c,d,e;if(null==g&&a.getComputedStyle){if(b=z.getElementsByTagName("body")[0],!b)return;c=z.createElement("div"),d=z.createElement("div"),c.style.cssText=i,b.appendC
 hild(c).appendChild(d),e=d.appendChild(z.createElement("div")),e.style.cssText=d.style.cssText=j,e.style.marginRight=e.style.width="0",d.style.width="1px",g=!parseFloat((a.getComputedStyle(e,null)||{}).marginRight),b.removeChild(c)}return g}});function k(){var b,c,h=z.getElementsByTagName("body")[0];h&&(b=z.createElement("div"),c=z.createElement("div"),b.style.cssText=i,h.appendChild(b).appendChild(c),c.style.cssText="-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;position:absolute;display:block;padding:1px;border:1px;width:4px;margin-top:1%;top:1%",n.swap(h,null!=h.style.zoom?{zoom:1}:{},function(){d=4===c.offsetWidth}),e=!0,f=!1,g=!0,a.getComputedStyle&&(f="1%"!==(a.getComputedStyle(c,null)||{}).top,e="4px"===(a.getComputedStyle(c,null)||{width:"4px"}).width),h.removeChild(b),c=h=null)}}(),n.swap=function(a,b,c,d){var e,f,g={};for(f in b)g[f]=a.style[f],a.style[f]=b[f];e=c.apply(a,d||[]);for(f in b)a.style[f]=g[f];return e};var Nb=/alpha\([^)]*\)/i,
 Ob=/opacity\s*=\s*([^)]*)/,Pb=/^(none|table(?!-c[ea]).+)/,Qb=new RegExp("^("+T+")(.*)$","i"),Rb=new RegExp("^([+-])=("+T+")","i"),Sb={position:"absolute",visibility:"hidden",display:"block"},Tb={letterSpacing:0,fontWeight:400},Ub=["Webkit","O","Moz","ms"];function Vb(a,b){if(b in a)return b;var c=b.charAt(0).toUpperCase()+b.slice(1),d=b,e=Ub.length;while(e--)if(b=Ub[e]+c,b in a)return b;return d}function Wb(a,b){for(var c,d,e,f=[],g=0,h=a.length;h>g;g++)d=a[g],d.style&&(f[g]=n._data(d,"olddisplay"),c=d.style.display,b?(f[g]||"none"!==c||(d.style.display=""),""===d.style.display&&V(d)&&(f[g]=n._data(d,"olddisplay",Gb(d.nodeName)))):f[g]||(e=V(d),(c&&"none"!==c||!e)&&n._data(d,"olddisplay",e?c:n.css(d,"display"))));for(g=0;h>g;g++)d=a[g],d.style&&(b&&"none"!==d.style.display&&""!==d.style.display||(d.style.display=b?f[g]||"":"none"));return a}function Xb(a,b,c){var d=Qb.exec(b);return d?Math.max(0,d[1]-(c||0))+(d[2]||"px"):b}function Yb(a,b,c,d,e){for(var f=c===(d?"border":"content")?
 4:"width"===b?1:0,g=0;4>f;f+=2)"margin"===c&&(g+=n.css(a,c+U[f],!0,e)),d?("content"===c&&(g-=n.css(a,"padding"+U[f],!0,e)),"margin"!==c&&(g-=n.css(a,"border"+U[f]+"Width",!0,e))):(g+=n.css(a,"padding"+U[f],!0,e),"padding"!==c&&(g+=n.css(a,"border"+U[f]+"Width",!0,e)));return g}function Zb(a,b,c){var d=!0,e="width"===b?a.offsetWidth:a.offsetHeight,f=Jb(a),g=l.boxSizing()&&"border-box"===n.css(a,"boxSizing",!1,f);if(0>=e||null==e){if(e=Kb(a,b,f),(0>e||null==e)&&(e=a.style[b]),Ib.test(e))return e;d=g&&(l.boxSizingReliable()||e===a.style[b]),e=parseFloat(e)||0}return e+Yb(a,b,c||(g?"border":"content"),d,f)+"px"}n.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=Kb(a,"opacity");return""===c?"1":c}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,order:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":l.cssFloat?"cssFloat":"styleFloat"},style:function(a,b,c,d){if(a&&3!==a.nodeType&&8!==a.nodeType&&a.style){var e,f,g,h=n.camelCase(b),i=a
 .style;if(b=n.cssProps[h]||(n.cssProps[h]=Vb(i,h)),g=n.cssHooks[b]||n.cssHooks[h],void 0===c)return g&&"get"in g&&void 0!==(e=g.get(a,!1,d))?e:i[b];if(f=typeof c,"string"===f&&(e=Rb.exec(c))&&(c=(e[1]+1)*e[2]+parseFloat(n.css(a,b)),f="number"),null!=c&&c===c&&("number"!==f||n.cssNumber[h]||(c+="px"),l.clearCloneStyle||""!==c||0!==b.indexOf("background")||(i[b]="inherit"),!(g&&"set"in g&&void 0===(c=g.set(a,c,d)))))try{i[b]="",i[b]=c}catch(j){}}},css:function(a,b,c,d){var e,f,g,h=n.camelCase(b);return b=n.cssProps[h]||(n.cssProps[h]=Vb(a.style,h)),g=n.cssHooks[b]||n.cssHooks[h],g&&"get"in g&&(f=g.get(a,!0,c)),void 0===f&&(f=Kb(a,b,d)),"normal"===f&&b in Tb&&(f=Tb[b]),""===c||c?(e=parseFloat(f),c===!0||n.isNumeric(e)?e||0:f):f}}),n.each(["height","width"],function(a,b){n.cssHooks[b]={get:function(a,c,d){return c?0===a.offsetWidth&&Pb.test(n.css(a,"display"))?n.swap(a,Sb,function(){return Zb(a,b,d)}):Zb(a,b,d):void 0},set:function(a,c,d){var e=d&&Jb(a);return Xb(a,c,d?Yb(a,b,d,l.boxSiz
 ing()&&"border-box"===n.css(a,"boxSizing",!1,e),e):0)}}}),l.opacity||(n.cssHooks.opacity={get:function(a,b){return Ob.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=n.isNumeric(b)?"alpha(opacity="+100*b+")":"",f=d&&d.filter||c.filter||"";c.zoom=1,(b>=1||""===b)&&""===n.trim(f.replace(Nb,""))&&c.removeAttribute&&(c.removeAttribute("filter"),""===b||d&&!d.filter)||(c.filter=Nb.test(f)?f.replace(Nb,e):f+" "+e)}}),n.cssHooks.marginRight=Mb(l.reliableMarginRight,function(a,b){return b?n.swap(a,{display:"inline-block"},Kb,[a,"marginRight"]):void 0}),n.each({margin:"",padding:"",border:"Width"},function(a,b){n.cssHooks[a+b]={expand:function(c){for(var d=0,e={},f="string"==typeof c?c.split(" "):[c];4>d;d++)e[a+U[d]+b]=f[d]||f[d-2]||f[0];return e}},Hb.test(a)||(n.cssHooks[a+b].set=Xb)}),n.fn.extend({css:function(a,b){return W(this,function(a,b,c){var d,e,f={},g=0;if(n.isArray(b)){for
 (d=Jb(a),e=b.length;e>g;g++)f[b[g]]=n.css(a,b[g],!1,d);return f}return void 0!==c?n.style(a,b,c):n.css(a,b)
-},a,b,arguments.length>1)},show:function(){return Wb(this,!0)},hide:function(){return Wb(this)},toggle:function(a){return"boolean"==typeof a?a?this.show():this.hide():this.each(function(){V(this)?n(this).show():n(this).hide()})}});function $b(a,b,c,d,e){return new $b.prototype.init(a,b,c,d,e)}n.Tween=$b,$b.prototype={constructor:$b,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||"swing",this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(n.cssNumber[c]?"":"px")},cur:function(){var a=$b.propHooks[this.prop];return a&&a.get?a.get(this):$b.propHooks._default.get(this)},run:function(a){var b,c=$b.propHooks[this.prop];return this.pos=b=this.options.duration?n.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):$b.propHooks._default.set(this),this}},$b.prototype.init.prototype=$b.prototype,$b.propHoo
 ks={_default:{get:function(a){var b;return null==a.elem[a.prop]||a.elem.style&&null!=a.elem.style[a.prop]?(b=n.css(a.elem,a.prop,""),b&&"auto"!==b?b:0):a.elem[a.prop]},set:function(a){n.fx.step[a.prop]?n.fx.step[a.prop](a):a.elem.style&&(null!=a.elem.style[n.cssProps[a.prop]]||n.cssHooks[a.prop])?n.style(a.elem,a.prop,a.now+a.unit):a.elem[a.prop]=a.now}}},$b.propHooks.scrollTop=$b.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},n.easing={linear:function(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2}},n.fx=$b.prototype.init,n.fx.step={};var _b,ac,bc=/^(?:toggle|show|hide)$/,cc=new RegExp("^(?:([+-])=|)("+T+")([a-z%]*)$","i"),dc=/queueHooks$/,ec=[jc],fc={"*":[function(a,b){var c=this.createTween(a,b),d=c.cur(),e=cc.exec(b),f=e&&e[3]||(n.cssNumber[a]?"":"px"),g=(n.cssNumber[a]||"px"!==f&&+d)&&cc.exec(n.css(c.elem,a)),h=1,i=20;if(g&&g[3]!==f){f=f||g[3],e=e||[],g=+d||1;do h=h||".5",g/=h,n.style(c.elem,a,g+f);while(h!==(h=
 c.cur()/d)&&1!==h&&--i)}return e&&(g=c.start=+g||+d||0,c.unit=f,c.end=e[1]?g+(e[1]+1)*e[2]:+e[2]),c}]};function gc(){return setTimeout(function(){_b=void 0}),_b=n.now()}function hc(a,b){var c,d={height:a},e=0;for(b=b?1:0;4>e;e+=2-b)c=U[e],d["margin"+c]=d["padding"+c]=a;return b&&(d.opacity=d.width=a),d}function ic(a,b,c){for(var d,e=(fc[b]||[]).concat(fc["*"]),f=0,g=e.length;g>f;f++)if(d=e[f].call(c,b,a))return d}function jc(a,b,c){var d,e,f,g,h,i,j,k,m=this,o={},p=a.style,q=a.nodeType&&V(a),r=n._data(a,"fxshow");c.queue||(h=n._queueHooks(a,"fx"),null==h.unqueued&&(h.unqueued=0,i=h.empty.fire,h.empty.fire=function(){h.unqueued||i()}),h.unqueued++,m.always(function(){m.always(function(){h.unqueued--,n.queue(a,"fx").length||h.empty.fire()})})),1===a.nodeType&&("height"in b||"width"in b)&&(c.overflow=[p.overflow,p.overflowX,p.overflowY],j=n.css(a,"display"),k=Gb(a.nodeName),"none"===j&&(j=k),"inline"===j&&"none"===n.css(a,"float")&&(l.inlineBlockNeedsLayout&&"inline"!==k?p.zoom=1:p.dis
 play="inline-block")),c.overflow&&(p.overflow="hidden",l.shrinkWrapBlocks()||m.always(function(){p.overflow=c.overflow[0],p.overflowX=c.overflow[1],p.overflowY=c.overflow[2]}));for(d in b)if(e=b[d],bc.exec(e)){if(delete b[d],f=f||"toggle"===e,e===(q?"hide":"show")){if("show"!==e||!r||void 0===r[d])continue;q=!0}o[d]=r&&r[d]||n.style(a,d)}if(!n.isEmptyObject(o)){r?"hidden"in r&&(q=r.hidden):r=n._data(a,"fxshow",{}),f&&(r.hidden=!q),q?n(a).show():m.done(function(){n(a).hide()}),m.done(function(){var b;n._removeData(a,"fxshow");for(b in o)n.style(a,b,o[b])});for(d in o)g=ic(q?r[d]:0,d,m),d in r||(r[d]=g.start,q&&(g.end=g.start,g.start="width"===d||"height"===d?1:0))}}function kc(a,b){var c,d,e,f,g;for(c in a)if(d=n.camelCase(c),e=b[d],f=a[c],n.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,delete a[c]),g=n.cssHooks[d],g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}function lc(a,b,c){var d,e,f=0,g=ec.length,h=n.Deferred().always(function(){
 delete i.elem}),i=function(){if(e)return!1;for(var b=_b||gc(),c=Math.max(0,j.startTime+j.duration-b),d=c/j.duration||0,f=1-d,g=0,i=j.tweens.length;i>g;g++)j.tweens[g].run(f);return h.notifyWith(a,[j,f,c]),1>f&&i?c:(h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:n.extend({},b),opts:n.extend(!0,{specialEasing:{}},c),originalProperties:b,originalOptions:c,startTime:_b||gc(),duration:c.duration,tweens:[],createTween:function(b,c){var d=n.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(d),d},stop:function(b){var c=0,d=b?j.tweens.length:0;if(e)return this;for(e=!0;d>c;c++)j.tweens[c].run(1);return b?h.resolveWith(a,[j,b]):h.rejectWith(a,[j,b]),this}}),k=j.props;for(kc(k,j.opts.specialEasing);g>f;f++)if(d=ec[f].call(j,a,k,j.opts))return d;return n.map(k,ic,j),n.isFunction(j.opts.start)&&j.opts.start.call(a,j),n.fx.timer(n.extend(i,{elem:a,anim:j,queue:j.opts.queue})),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(
 j.opts.always)}n.Animation=n.extend(lc,{tweener:function(a,b){n.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");for(var c,d=0,e=a.length;e>d;d++)c=a[d],fc[c]=fc[c]||[],fc[c].unshift(b)},prefilter:function(a,b){b?ec.unshift(a):ec.push(a)}}),n.speed=function(a,b,c){var d=a&&"object"==typeof a?n.extend({},a):{complete:c||!c&&b||n.isFunction(a)&&a,duration:a,easing:c&&b||b&&!n.isFunction(b)&&b};return d.duration=n.fx.off?0:"number"==typeof d.duration?d.duration:d.duration in n.fx.speeds?n.fx.speeds[d.duration]:n.fx.speeds._default,(null==d.queue||d.queue===!0)&&(d.queue="fx"),d.old=d.complete,d.complete=function(){n.isFunction(d.old)&&d.old.call(this),d.queue&&n.dequeue(this,d.queue)},d},n.fn.extend({fadeTo:function(a,b,c,d){return this.filter(V).css("opacity",0).show().end().animate({opacity:b},a,c,d)},animate:function(a,b,c,d){var e=n.isEmptyObject(a),f=n.speed(b,c,d),g=function(){var b=lc(this,n.extend({},a),f);(e||n._data(this,"finish"))&&b.stop(!0)};return g.finish=g,e||f.queue===!1?thi
 s.each(g):this.queue(f.queue,g)},stop:function(a,b,c){var d=function(a){var b=a.stop;delete a.stop,b(c)};return"string"!=typeof a&&(c=b,b=a,a=void 0),b&&a!==!1&&this.queue(a||"fx",[]),this.each(function(){var b=!0,e=null!=a&&a+"queueHooks",f=n.timers,g=n._data(this);if(e)g[e]&&g[e].stop&&d(g[e]);else for(e in g)g[e]&&g[e].stop&&dc.test(e)&&d(g[e]);for(e=f.length;e--;)f[e].elem!==this||null!=a&&f[e].queue!==a||(f[e].anim.stop(c),b=!1,f.splice(e,1));(b||!c)&&n.dequeue(this,a)})},finish:function(a){return a!==!1&&(a=a||"fx"),this.each(function(){var b,c=n._data(this),d=c[a+"queue"],e=c[a+"queueHooks"],f=n.timers,g=d?d.length:0;for(c.finish=!0,n.queue(this,a,[]),e&&e.stop&&e.stop.call(this,!0),b=f.length;b--;)f[b].elem===this&&f[b].queue===a&&(f[b].anim.stop(!0),f.splice(b,1));for(b=0;g>b;b++)d[b]&&d[b].finish&&d[b].finish.call(this);delete c.finish})}}),n.each(["toggle","show","hide"],function(a,b){var c=n.fn[b];n.fn[b]=function(a,d,e){return null==a||"boolean"==typeof a?c.apply(this,a
 rguments):this.animate(hc(b,!0),a,d,e)}}),n.each({slideDown:hc("show"),slideUp:hc("hide"),slideToggle:hc("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(a,b){n.fn[a]=function(a,c,d){return this.animate(b,a,c,d)}}),n.timers=[],n.fx.tick=function(){var a,b=n.timers,c=0;for(_b=n.now();c<b.length;c++)a=b[c],a()||b[c]!==a||b.splice(c--,1);b.length||n.fx.stop(),_b=void 0},n.fx.timer=function(a){n.timers.push(a),a()?n.fx.start():n.timers.pop()},n.fx.interval=13,n.fx.start=function(){ac||(ac=setInterval(n.fx.tick,n.fx.interval))},n.fx.stop=function(){clearInterval(ac),ac=null},n.fx.speeds={slow:600,fast:200,_default:400},n.fn.delay=function(a,b){return a=n.fx?n.fx.speeds[a]||a:a,b=b||"fx",this.queue(b,function(b,c){var d=setTimeout(b,a);c.stop=function(){clearTimeout(d)}})},function(){var a,b,c,d,e=z.createElement("div");e.setAttribute("className","t"),e.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",a=e.getE
 lementsByTagName("a")[0],c=z.createElement("select"),d=c.appendChild(z.createElement("option")),b=e.getElementsByTagName("input")[0],a.style.cssText="top:1px",l.getSetAttribute="t"!==e.className,l.style=/top/.test(a.getAttribute("style")),l.hrefNormalized="/a"===a.getAttribute("href"),l.checkOn=!!b.value,l.optSelected=d.selected,l.enctype=!!z.createElement("form").enctype,c.disabled=!0,l.optDisabled=!d.disabled,b=z.createElement("input"),b.setAttribute("value",""),l.input=""===b.getAttribute("value"),b.value="t",b.setAttribute("type","radio"),l.radioValue="t"===b.value,a=b=c=d=e=null}();var mc=/\r/g;n.fn.extend({val:function(a){var b,c,d,e=this[0];{if(arguments.length)return d=n.isFunction(a),this.each(function(c){var e;1===this.nodeType&&(e=d?a.call(this,c,n(this).val()):a,null==e?e="":"number"==typeof e?e+="":n.isArray(e)&&(e=n.map(e,function(a){return null==a?"":a+""})),b=n.valHooks[this.type]||n.valHooks[this.nodeName.toLowerCase()],b&&"set"in b&&void 0!==b.set(this,e,"value")||
 (this.value=e))});if(e)return b=n.valHooks[e.type]||n.valHooks[e.nodeName.toLowerCase()],b&&"get"in b&&void 0!==(c=b.get(e,"value"))?c:(c=e.value,"string"==typeof c?c.replace(mc,""):null==c?"":c)}}}),n.extend({valHooks:{option:{get:function(a){var b=n.find.attr(a,"value");return null!=b?b:n.text(a)}},select:{get:function(a){for(var b,c,d=a.options,e=a.selectedIndex,f="select-one"===a.type||0>e,g=f?null:[],h=f?e+1:d.length,i=0>e?h:f?e:0;h>i;i++)if(c=d[i],!(!c.selected&&i!==e||(l.optDisabled?c.disabled:null!==c.getAttribute("disabled"))||c.parentNode.disabled&&n.nodeName(c.parentNode,"optgroup"))){if(b=n(c).val(),f)return b;g.push(b)}return g},set:function(a,b){var c,d,e=a.options,f=n.makeArray(b),g=e.length;while(g--)if(d=e[g],n.inArray(n.valHooks.option.get(d),f)>=0)try{d.selected=c=!0}catch(h){d.scrollHeight}else d.selected=!1;return c||(a.selectedIndex=-1),e}}}}),n.each(["radio","checkbox"],function(){n.valHooks[this]={set:function(a,b){return n.isArray(b)?a.checked=n.inArray(n(a)
 .val(),b)>=0:void 0}},l.checkOn||(n.valHooks[this].get=function(a){return null===a.getAttribute("value")?"on":a.value})});var nc,oc,pc=n.expr.attrHandle,qc=/^(?:checked|selected)$/i,rc=l.getSetAttribute,sc=l.input;n.fn.extend({attr:function(a,b){return W(this,n.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){n.removeAttr(this,a)})}}),n.extend({attr:function(a,b,c){var d,e,f=a.nodeType;if(a&&3!==f&&8!==f&&2!==f)return typeof a.getAttribute===L?n.prop(a,b,c):(1===f&&n.isXMLDoc(a)||(b=b.toLowerCase(),d=n.attrHooks[b]||(n.expr.match.bool.test(b)?oc:nc)),void 0===c?d&&"get"in d&&null!==(e=d.get(a,b))?e:(e=n.find.attr(a,b),null==e?void 0:e):null!==c?d&&"set"in d&&void 0!==(e=d.set(a,c,b))?e:(a.setAttribute(b,c+""),c):void n.removeAttr(a,b))},removeAttr:function(a,b){var c,d,e=0,f=b&&b.match(F);if(f&&1===a.nodeType)while(c=f[e++])d=n.propFix[c]||c,n.expr.match.bool.test(c)?sc&&rc||!qc.test(c)?a[d]=!1:a[n.camelCase("default-"+c)]=a[d]=!1:n.attr(a,c,""),a.rem
 oveAttribute(rc?c:d)},attrHooks:{type:{set:function(a,b){if(!l.radioValue&&"radio"===b&&n.nodeName(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}}}}),oc={set:function(a,b,c){return b===!1?n.removeAttr(a,c):sc&&rc||!qc.test(c)?a.setAttribute(!rc&&n.propFix[c]||c,c):a[n.camelCase("default-"+c)]=a[c]=!0,c}},n.each(n.expr.match.bool.source.match(/\w+/g),function(a,b){var c=pc[b]||n.find.attr;pc[b]=sc&&rc||!qc.test(b)?function(a,b,d){var e,f;return d||(f=pc[b],pc[b]=e,e=null!=c(a,b,d)?b.toLowerCase():null,pc[b]=f),e}:function(a,b,c){return c?void 0:a[n.camelCase("default-"+b)]?b.toLowerCase():null}}),sc&&rc||(n.attrHooks.value={set:function(a,b,c){return n.nodeName(a,"input")?void(a.defaultValue=b):nc&&nc.set(a,b,c)}}),rc||(nc={set:function(a,b,c){var d=a.getAttributeNode(c);return d||a.setAttributeNode(d=a.ownerDocument.createAttribute(c)),d.value=b+="","value"===c||b===a.getAttribute(c)?b:void 0}},pc.id=pc.name=pc.coords=function(a,b,c){var d;return c?void
  0:(d=a.getAttributeNode(b))&&""!==d.value?d.value:null},n.valHooks.button={get:function(a,b){var c=a.getAttributeNode(b);return c&&c.specified?c.value:void 0},set:nc.set},n.attrHooks.contenteditable={set:function(a,b,c){nc.set(a,""===b?!1:b,c)}},n.each(["width","height"],function(a,b){n.attrHooks[b]={set:function(a,c){return""===c?(a.setAttribute(b,"auto"),c):void 0}}})),l.style||(n.attrHooks.style={get:function(a){return a.style.cssText||void 0},set:function(a,b){return a.style.cssText=b+""}});var tc=/^(?:input|select|textarea|button|object)$/i,uc=/^(?:a|area)$/i;n.fn.extend({prop:function(a,b){return W(this,n.prop,a,b,arguments.length>1)},removeProp:function(a){return a=n.propFix[a]||a,this.each(function(){try{this[a]=void 0,delete this[a]}catch(b){}})}}),n.extend({propFix:{"for":"htmlFor","class":"className"},prop:function(a,b,c){var d,e,f,g=a.nodeType;if(a&&3!==g&&8!==g&&2!==g)return f=1!==g||!n.isXMLDoc(a),f&&(b=n.propFix[b]||b,e=n.propHooks[b]),void 0!==c?e&&"set"in e&&void 0
 !==(d=e.set(a,c,b))?d:a[b]=c:e&&"get"in e&&null!==(d=e.get(a,b))?d:a[b]},propHooks:{tabIndex:{get:function(a){var b=n.find.attr(a,"tabindex");return b?parseInt(b,10):tc.test(a.nodeName)||uc.test(a.nodeName)&&a.href?0:-1}}}}),l.hrefNormalized||n.each(["href","src"],function(a,b){n.propHooks[b]={get:function(a){return a.getAttribute(b,4)}}}),l.optSelected||(n.propHooks.selected={get:function(a){var b=a.parentNode;return b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex),null}}),n.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"],function(){n.propFix[this.toLowerCase()]=this}),l.enctype||(n.propFix.enctype="encoding");var vc=/[\t\r\n\f]/g;n.fn.extend({addClass:function(a){var b,c,d,e,f,g,h=0,i=this.length,j="string"==typeof a&&a;if(n.isFunction(a))return this.each(function(b){n(this).addClass(a.call(this,b,this.className))});if(j)for(b=(a||"").match(F)||[];i>h;h++)if(c=this[h],d=1===c.nodeType&&(
 c.className?(" "+c.className+" ").replace(vc," "):" ")){f=0;while(e=b[f++])d.indexOf(" "+e+" ")<0&&(d+=e+" ");g=n.trim(d),c.className!==g&&(c.className=g)}return this},removeClass:function(a){var b,c,d,e,f,g,h=0,i=this.length,j=0===arguments.length||"string"==typeof a&&a;if(n.isFunction(a))return this.each(function(b){n(this).removeClass(a.call(this,b,this.className))});if(j)for(b=(a||"").match(F)||[];i>h;h++)if(c=this[h],d=1===c.nodeType&&(c.className?(" "+c.className+" ").replace(vc," "):"")){f=0;while(e=b[f++])while(d.indexOf(" "+e+" ")>=0)d=d.replace(" "+e+" "," ");g=a?n.trim(d):"",c.className!==g&&(c.className=g)}return this},toggleClass:function(a,b){var c=typeof a;return"boolean"==typeof b&&"string"===c?b?this.addClass(a):this.removeClass(a):this.each(n.isFunction(a)?function(c){n(this).toggleClass(a.call(this,c,this.className,b),b)}:function(){if("string"===c){var b,d=0,e=n(this),f=a.match(F)||[];while(b=f[d++])e.hasClass(b)?e.removeClass(b):e.addClass(b)}else(c===L||"boolea
 n"===c)&&(this.className&&n._data(this,"__className__",this.className),this.className=this.className||a===!1?"":n._data(this,"__className__")||"")})},hasClass:function(a){for(var b=" "+a+" ",c=0,d=this.length;d>c;c++)if(1===this[c].nodeType&&(" "+this[c].className+" ").replace(vc," ").indexOf(b)>=0)return!0;return!1}}),n.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(a,b){n.fn[b]=function(a,c){return arguments.length>0?this.on(b,null,a,c):this.trigger(b)}}),n.fn.extend({hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)},bind:function(a,b,c){return this.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return 1===arguments.length?this.off(a,"**"):this.off(b,a||"**",c)}});var wc=n.now(),xc=/\?/,yc=/(,)|(
 \[|{)|(}|])|"(?:[^"\\\r\n]|\\["\\\/bfnrt]|\\u[\da-fA-F]{4})*"\s*:?|true|false|null|-?(?!0\d)\d+(?:\.\d+|)(?:[eE][+-]?\d+|)/g;n.parseJSON=function(b){if(a.JSON&&a.JSON.parse)return a.JSON.parse(b+"");var c,d=null,e=n.trim(b+"");return e&&!n.trim(e.replace(yc,function(a,b,e,f){return c&&b&&(d=0),0===d?a:(c=e||b,d+=!f-!e,"")}))?Function("return "+e)():n.error("Invalid JSON: "+b)},n.parseXML=function(b){var c,d;if(!b||"string"!=typeof b)return null;try{a.DOMParser?(d=new DOMParser,c=d.parseFromString(b,"text/xml")):(c=new ActiveXObject("Microsoft.XMLDOM"),c.async="false",c.loadXML(b))}catch(e){c=void 0}return c&&c.documentElement&&!c.getElementsByTagName("parsererror").length||n.error("Invalid XML: "+b),c};var zc,Ac,Bc=/#.*$/,Cc=/([?&])_=[^&]*/,Dc=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Ec=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Fc=/^(?:GET|HEAD)$/,Gc=/^\/\//,Hc=/^([\w.+-]+:)(?:\/\/(?:[^\/?#]*@|)([^\/?#:]*)(?::(\d+)|)|)/,Ic={},Jc={},Kc="*/".concat("*");try{Ac=location.href}c
 atch(Lc){Ac=z.createElement("a"),Ac.href="",Ac=Ac.href}zc=Hc.exec(Ac.toLowerCase())||[];function Mc(a){return function(b,c){"string"!=typeof b&&(c=b,b="*");var d,e=0,f=b.toLowerCase().match(F)||[];if(n.isFunction(c))while(d=f[e++])"+"===d.charAt(0)?(d=d.slice(1)||"*",(a[d]=a[d]||[]).unshift(c)):(a[d]=a[d]||[]).push(c)}}function Nc(a,b,c,d){var e={},f=a===Jc;function g(h){var i;return e[h]=!0,n.each(a[h]||[],function(a,h){var j=h(b,c,d);return"string"!=typeof j||f||e[j]?f?!(i=j):void 0:(b.dataTypes.unshift(j),g(j),!1)}),i}return g(b.dataTypes[0])||!e["*"]&&g("*")}function Oc(a,b){var c,d,e=n.ajaxSettings.flatOptions||{};for(d in b)void 0!==b[d]&&((e[d]?a:c||(c={}))[d]=b[d]);return c&&n.extend(!0,a,c),a}function Pc(a,b,c){var d,e,f,g,h=a.contents,i=a.dataTypes;while("*"===i[0])i.shift(),void 0===e&&(e=a.mimeType||b.getResponseHeader("Content-Type"));if(e)for(g in h)if(h[g]&&h[g].test(e)){i.unshift(g);break}if(i[0]in c)f=i[0];else{for(g in c){if(!i[0]||a.converters[g+" "+i[0]]){f=g;bre
 ak}d||(d=g)}f=f||d}return f?(f!==i[0]&&i.unshift(f),c[f]):void 0}function Qc(a,b,c,d){var e,f,g,h,i,j={},k=a.dataTypes.slice();if(k[1])for(g in a.converters)j[g.toLowerCase()]=a.converters[g];f=k.shift();while(f)if(a.responseFields[f]&&(c[a.responseFields[f]]=b),!i&&d&&a.dataFilter&&(b=a.dataFilter(b,a.dataType)),i=f,f=k.shift())if("*"===f)f=i;else if("*"!==i&&i!==f){if(g=j[i+" "+f]||j["* "+f],!g)for(e in j)if(h=e.split(" "),h[1]===f&&(g=j[i+" "+h[0]]||j["* "+h[0]])){g===!0?g=j[e]:j[e]!==!0&&(f=h[0],k.unshift(h[1]));break}if(g!==!0)if(g&&a["throws"])b=g(b);else try{b=g(b)}catch(l){return{state:"parsererror",error:g?l:"No conversion from "+i+" to "+f}}}return{state:"success",data:b}}n.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:Ac,type:"GET",isLocal:Ec.test(zc[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Kc,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text
 /javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":n.parseJSON,"text xml":n.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(a,b){return b?Oc(Oc(a,n.ajaxSettings),b):Oc(n.ajaxSettings,a)},ajaxPrefilter:Mc(Ic),ajaxTransport:Mc(Jc),ajax:function(a,b){"object"==typeof a&&(b=a,a=void 0),b=b||{};var c,d,e,f,g,h,i,j,k=n.ajaxSetup({},b),l=k.context||k,m=k.context&&(l.nodeType||l.jquery)?n(l):n.event,o=n.Deferred(),p=n.Callbacks("once memory"),q=k.statusCode||{},r={},s={},t=0,u="canceled",v={readyState:0,getResponseHeader:function(a){var b;if(2===t){if(!j){j={};while(b=Dc.exec(f))j[b[1].toLowerCase()]=b[2]}b=j[a.toLowerCase()]}return null==b?null:b},getAllResponseHeaders:function(){return 2===t?f:null},setRequestHeader:function(a,b){var c=a.toLowerCase();return t||(a=s[c]=s[c]||a,r[a]=b),this},overrideMimeType:function(a){return t||(k.mimeTyp
 e=a),this},statusCode:function(a){var b;if(a)if(2>t)for(b in a)q[b]=[q[b],a[b]];else v.always(a[v.status]);return this},abort:function(a){var b=a||u;return i&&i.abort(b),x(0,b),this}};if(o.promise(v).complete=p.add,v.success=v.done,v.error=v.fail,k.url=((a||k.url||Ac)+"").replace(Bc,"").replace(Gc,zc[1]+"//"),k.type=b.method||b.type||k.method||k.type,k.dataTypes=n.trim(k.dataType||"*").toLowerCase().match(F)||[""],null==k.crossDomain&&(c=Hc.exec(k.url.toLowerCase()),k.crossDomain=!(!c||c[1]===zc[1]&&c[2]===zc[2]&&(c[3]||("http:"===c[1]?"80":"443"))===(zc[3]||("http:"===zc[1]?"80":"443")))),k.data&&k.processData&&"string"!=typeof k.data&&(k.data=n.param(k.data,k.traditional)),Nc(Ic,k,b,v),2===t)return v;h=k.global,h&&0===n.active++&&n.event.trigger("ajaxStart"),k.type=k.type.toUpperCase(),k.hasContent=!Fc.test(k.type),e=k.url,k.hasContent||(k.data&&(e=k.url+=(xc.test(e)?"&":"?")+k.data,delete k.data),k.cache===!1&&(k.url=Cc.test(e)?e.replace(Cc,"$1_="+wc++):e+(xc.test(e)?"&":"?")+"_=
 "+wc++)),k.ifModified&&(n.lastModified[e]&&v.setRequestHeader("If-Modified-Since",n.lastModified[e]),n.etag[e]&&v.setRequestHeader("If-None-Match",n.etag[e])),(k.data&&k.hasContent&&k.contentType!==!1||b.contentType)&&v.setRequestHeader("Content-Type",k.contentType),v.setRequestHeader("Accept",k.dataTypes[0]&&k.accepts[k.dataTypes[0]]?k.accepts[k.dataTypes[0]]+("*"!==k.dataTypes[0]?", "+Kc+"; q=0.01":""):k.accepts["*"]);for(d in k.headers)v.setRequestHeader(d,k.headers[d]);if(k.beforeSend&&(k.beforeSend.call(l,v,k)===!1||2===t))return v.abort();u="abort";for(d in{success:1,error:1,complete:1})v[d](k[d]);if(i=Nc(Jc,k,b,v)){v.readyState=1,h&&m.trigger("ajaxSend",[v,k]),k.async&&k.timeout>0&&(g=setTimeout(function(){v.abort("timeout")},k.timeout));try{t=1,i.send(r,x)}catch(w){if(!(2>t))throw w;x(-1,w)}}else x(-1,"No Transport");function x(a,b,c,d){var j,r,s,u,w,x=b;2!==t&&(t=2,g&&clearTimeout(g),i=void 0,f=d||"",v.readyState=a>0?4:0,j=a>=200&&300>a||304===a,c&&(u=Pc(k,v,c)),u=Qc(k,u,v,
 j),j?(k.ifModified&&(w=v.getResponseHeader("Last-Modified"),w&&(n.lastModified[e]=w),w=v.getResponseHeader("etag"),w&&(n.etag[e]=w)),204===a||"HEAD"===k.type?x="nocontent":304===a?x="notmodified":(x=u.state,r=u.data,s=u.error,j=!s)):(s=x,(a||!x)&&(x="error",0>a&&(a=0))),v.status=a,v.statusText=(b||x)+"",j?o.resolveWith(l,[r,x,v]):o.rejectWith(l,[v,x,s]),v.statusCode(q),q=void 0,h&&m.trigger(j?"ajaxSuccess":"ajaxError",[v,k,j?r:s]),p.fireWith(l,[v,x]),h&&(m.trigger("ajaxComplete",[v,k]),--n.active||n.event.trigger("ajaxStop")))}return v},getJSON:function(a,b,c){return n.get(a,b,c,"json")},getScript:function(a,b){return n.get(a,void 0,b,"script")}}),n.each(["get","post"],function(a,b){n[b]=function(a,c,d,e){return n.isFunction(c)&&(e=e||d,d=c,c=void 0),n.ajax({url:a,type:b,dataType:e,data:c,success:d})}}),n.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(a,b){n.fn[b]=function(a){return this.on(b,a)}}),n._evalUrl=function(a){return n.ajax({url
 :a,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0})},n.fn.extend({wrapAll:function(a){if(n.isFunction(a))return this.each(function(b){n(this).wrapAll(a.call(this,b))});if(this[0]){var b=n(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&1===a.firstChild.nodeType)a=a.firstChild;return a}).append(this)}return this},wrapInner:function(a){return this.each(n.isFunction(a)?function(b){n(this).wrapInner(a.call(this,b))}:function(){var b=n(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=n.isFunction(a);return this.each(function(c){n(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){n.nodeName(this,"body")||n(this).replaceWith(this.childNodes)}).end()}}),n.expr.filters.hidden=function(a){return a.offsetWidth<=0&&a.offsetHeight<=0||!l.reliableHiddenOffsets()&&"none"===(a.style&&a.style.display||n.css(a,"display"))},n.expr.f
 ilters.visible=function(a){return!n.expr.filters.hidden(a)};var Rc=/%20/g,Sc=/\[\]$/,Tc=/\r?\n/g,Uc=/^(?:submit|button|image|reset|file)$/i,Vc=/^(?:input|select|textarea|keygen)/i;function Wc(a,b,c,d){var e;if(n.isArray(b))n.each(b,function(b,e){c||Sc.test(a)?d(a,e):Wc(a+"["+("object"==typeof e?b:"")+"]",e,c,d)});else if(c||"object"!==n.type(b))d(a,b);else for(e in b)Wc(a+"["+e+"]",b[e],c,d)}n.param=function(a,b){var c,d=[],e=function(a,b){b=n.isFunction(b)?b():null==b?"":b,d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};if(void 0===b&&(b=n.ajaxSettings&&n.ajaxSettings.traditional),n.isArray(a)||a.jquery&&!n.isPlainO

<TRUNCATED>

[11/13] drill git commit: DRILL-5699: Drill Web UI Page Source Has Links To External Sites

Posted by jn...@apache.org.
DRILL-5699: Drill Web UI Page Source Has Links To External Sites

close #891


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/1cec10b9
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/1cec10b9
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/1cec10b9

Branch: refs/heads/master
Commit: 1cec10b957b13c1006bb644a44a833d81da5a187
Parents: fd7fba6
Author: Sindhuri Rayavaram <si...@srayava-E422.local>
Authored: Mon Jul 31 11:45:53 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/rest/generic.ftl         |   4 +-
 .../src/main/resources/rest/query/result.ftl    |  13 +-
 .../static/css/dataTables.colVis-1.1.0.min.css  |   1 +
 .../rest/static/css/dataTables.jqueryui.css     | 414 +++++++++++++++++++
 .../css/images/ui-bg_glass_75_e6e6e6_1x400.png  | Bin 0 -> 110 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    | Bin 0 -> 101 bytes
 .../css/images/ui-icons_888888_256x240.png      | Bin 0 -> 4369 bytes
 .../rest/static/css/jquery-ui-1.10.3.min.css    |   9 +
 .../static/js/dataTables.colVis-1.1.0.min.js    |  23 ++
 .../rest/static/js/jquery-3.2.1.min.js          |   4 +
 .../static/js/jquery.dataTables-1.10.0.min.js   | 152 +++++++
 .../main/resources/rest/static/js/jquery.min.js |   4 -
 12 files changed, 611 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/generic.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/generic.ftl b/exec/java-exec/src/main/resources/rest/generic.ftl
index 60869e7..9025adb 100644
--- a/exec/java-exec/src/main/resources/rest/generic.ftl
+++ b/exec/java-exec/src/main/resources/rest/generic.ftl
@@ -27,7 +27,9 @@
 
       <link href="/static/css/bootstrap.min.css" rel="stylesheet">
 
-      <script src="/static/js/jquery.min.js"></script>
+      <script type="text/javascript" language="javascript" src="//ajax.googleapis.com/ajax/libs/jquery/3.2.1/jquery.min.js"></script>
+      <script> window.jQuery  || document.write('<script type="text/javascript" language="javascript" src="../static/js/jquery-3.2.1.min.js">\x3C/script>')
+      </script>
       <script src="/static/js/bootstrap.min.js"></script>
 
       <!-- HTML5 shim and Respond.js IE8 support of HTML5 elements and media queries -->

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/query/result.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/query/result.ftl b/exec/java-exec/src/main/resources/rest/query/result.ftl
index 7fe52a4..8382e4f 100644
--- a/exec/java-exec/src/main/resources/rest/query/result.ftl
+++ b/exec/java-exec/src/main/resources/rest/query/result.ftl
@@ -11,14 +11,11 @@
 
 <#include "*/generic.ftl">
 <#macro page_head>
-  <link rel="stylesheet" type="text/css" href="//code.jquery.com/ui/1.10.3/themes/smoothness/jquery-ui.css">
-  <link rel="stylesheet" type="text/css" href="//cdn.datatables.net/plug-ins/be7019ee387/integration/jqueryui/dataTables.jqueryui.css">
-
-  <script type="text/javascript" language="javascript" src="//code.jquery.com/jquery-1.10.2.min.js"></script>
-  <script type="text/javascript" language="javascript" src="//cdn.datatables.net/1.10.0/js/jquery.dataTables.js"></script>
-
-  <link rel="stylesheet" type="text/css" href="//cdn.datatables.net/colvis/1.1.0/css/dataTables.colVis.css">
-  <script type="text/javascript" language="javascript" src="//cdn.datatables.net/colvis/1.1.0/js/dataTables.colVis.min.js"></script>
+    <script type="text/javascript" language="javascript"  src="../static/js/jquery.dataTables-1.10.0.min.js"> </script>
+    <script type="text/javascript" language="javascript" src="../static/js/dataTables.colVis-1.1.0.min.js"></script>
+    <link href="/static/css/dataTables.colVis-1.1.0.min.css" rel="stylesheet">
+    <link href="/static/css/dataTables.jqueryui.css" rel="stylesheet">
+    <link href="/static/css/jquery-ui-1.10.3.min.css" rel="stylesheet">
 </#macro>
 
 <#macro page_body>

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/dataTables.colVis-1.1.0.min.css
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/dataTables.colVis-1.1.0.min.css b/exec/java-exec/src/main/resources/rest/static/css/dataTables.colVis-1.1.0.min.css
new file mode 100644
index 0000000..64f1507
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/css/dataTables.colVis-1.1.0.min.css
@@ -0,0 +1 @@
+div.ColVis{float:right;margin-bottom:1em}button.ColVis_Button,ul.ColVis_collection li{position:relative;float:left;margin-right:3px;padding:5px 8px;border:1px solid #999;cursor:pointer;*cursor:hand;font-size:0.88em;color:black !important;white-space:nowrap;-webkit-border-radius:2px;-moz-border-radius:2px;-ms-border-radius:2px;-o-border-radius:2px;border-radius:2px;-webkit-box-shadow:1px 1px 3px #ccc;-moz-box-shadow:1px 1px 3px #ccc;-ms-box-shadow:1px 1px 3px #ccc;-o-box-shadow:1px 1px 3px #ccc;box-shadow:1px 1px 3px #ccc;background:#ffffff;background:-webkit-linear-gradient(top, #fff 0%, #f3f3f3 89%, #f9f9f9 100%);background:-moz-linear-gradient(top, #fff 0%, #f3f3f3 89%, #f9f9f9 100%);background:-ms-linear-gradient(top, #fff 0%, #f3f3f3 89%, #f9f9f9 100%);background:-o-linear-gradient(top, #fff 0%, #f3f3f3 89%, #f9f9f9 100%);background:linear-gradient(top, #fff 0%, #f3f3f3 89%, #f9f9f9 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffff', endColorstr='#f9
 f9f9',GradientType=0 )}.ColVis_Button:hover,ul.ColVis_collection li:hover{border:1px solid #666;text-decoration:none !important;-webkit-box-shadow:1px 1px 3px #999;-moz-box-shadow:1px 1px 3px #999;-ms-box-shadow:1px 1px 3px #999;-o-box-shadow:1px 1px 3px #999;box-shadow:1px 1px 3px #999;background:#f3f3f3;background:-webkit-linear-gradient(top, #f3f3f3 0%, #e2e2e2 89%, #f4f4f4 100%);background:-moz-linear-gradient(top, #f3f3f3 0%, #e2e2e2 89%, #f4f4f4 100%);background:-ms-linear-gradient(top, #f3f3f3 0%, #e2e2e2 89%, #f4f4f4 100%);background:-o-linear-gradient(top, #f3f3f3 0%, #e2e2e2 89%, #f4f4f4 100%);background:linear-gradient(top, #f3f3f3 0%, #e2e2e2 89%, #f4f4f4 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#f3f3f3', endColorstr='#f4f4f4',GradientType=0 )}button.ColVis_Button{height:30px;padding:3px 8px}button.ColVis_Button::-moz-focus-inner{border:none !important;padding:0}button.ColVis_Button:active{outline:none}div.ColVis_collectionBackground{positio
 n:fixed;top:0;left:0;height:100%;width:100%;background-color:black;z-index:1100}ul.ColVis_collection{width:150px;padding:8px 8px 4px 8px;margin:0;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.4);background-color:#f3f3f3;background-color:rgba(255,255,255,0.3);overflow:hidden;z-index:2002;-webkit-border-radius:5px;-moz-border-radius:5px;-ms-border-radius:5px;-o-border-radius:5px;border-radius:5px;-webkit-box-shadow:3px 3px 5px rgba(0,0,0,0.3);-moz-box-shadow:3px 3px 5px rgba(0,0,0,0.3);-ms-box-shadow:3px 3px 5px rgba(0,0,0,0.3);-o-box-shadow:3px 3px 5px rgba(0,0,0,0.3);box-shadow:3px 3px 5px rgba(0,0,0,0.3)}ul.ColVis_collection li{position:relative;height:auto;left:0;right:0;padding:0.5em;display:block;float:none;margin-bottom:4px;-webkit-box-shadow:1px 1px 3px #999;-moz-box-shadow:1px 1px 3px #999;-ms-box-shadow:1px 1px 3px #999;-o-box-shadow:1px 1px 3px #999;box-shadow:1px 1px 3px #999}ul.ColVis_collection li{text-align:left}ul.ColVis_collection li.ColVis_Button:hover{border:1
 px solid #999;background-color:#f0f0f0}ul.ColVis_collection li span{display:inline-block;padding-left:0.5em;cursor:pointer}ul.ColVis_collection li.ColVis_Special{border-color:#555;background:#ededed;background:-webkit-linear-gradient(top, #ededed 0%, #d6d6d6 77%, #e8e8e8 100%);background:-moz-linear-gradient(top, #ededed 0%, #d6d6d6 77%, #e8e8e8 100%);background:-ms-linear-gradient(top, #ededed 0%, #d6d6d6 77%, #e8e8e8 100%);background:-o-linear-gradient(top, #ededed 0%, #d6d6d6 77%, #e8e8e8 100%);background:linear-gradient(to bottom, #ededed 0%, #d6d6d6 77%, #e8e8e8 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ededed', endColorstr='#e8e8e8',GradientType=0 )}ul.ColVis_collection li.ColVis_Special:hover{background:#e2e2e2;background:-webkit-linear-gradient(top, #d0d0d0 0%, #d5d5d5 89%, #e2e2e2 100%);background:-moz-linear-gradient(top, #d0d0d0 0%, #d5d5d5 89%, #e2e2e2 100%);background:-ms-linear-gradient(top, #d0d0d0 0%, #d5d5d5 89%, #e2e2e2 100%);backgroun
 d:-o-linear-gradient(top, #d0d0d0 0%, #d5d5d5 89%, #e2e2e2 100%);background:linear-gradient(top, #d0d0d0 0%, #d5d5d5 89%, #e2e2e2 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#f3f3f3', endColorstr='#e2e2e2',GradientType=0 )}span.ColVis_radio{display:inline-block;width:20px}div.ColVis_catcher{position:absolute;z-index:1101}.disabled{color:#999}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/dataTables.jqueryui.css
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/dataTables.jqueryui.css b/exec/java-exec/src/main/resources/rest/static/css/dataTables.jqueryui.css
new file mode 100644
index 0000000..254309a
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/css/dataTables.jqueryui.css
@@ -0,0 +1,414 @@
+/*
+ * Table styles
+ */
+table.dataTable {
+  width: 100%;
+  margin: 0 auto;
+  clear: both;
+  border-collapse: separate;
+  border-spacing: 0;
+  /*
+   * Header and footer styles
+   */
+  /*
+   * Body styles
+   */
+}
+table.dataTable thead th,
+table.dataTable thead td,
+table.dataTable tfoot th,
+table.dataTable tfoot td {
+  padding: 4px 10px;
+}
+table.dataTable thead th,
+table.dataTable tfoot th {
+  font-weight: bold;
+}
+table.dataTable thead th:active,
+table.dataTable thead td:active {
+  outline: none;
+}
+table.dataTable thead .sorting_asc,
+table.dataTable thead .sorting_desc,
+table.dataTable thead .sorting {
+  cursor: pointer;
+  *cursor: hand;
+}
+table.dataTable thead th div.DataTables_sort_wrapper {
+  position: relative;
+  padding-right: 10px;
+}
+table.dataTable thead th div.DataTables_sort_wrapper span {
+  position: absolute;
+  top: 50%;
+  margin-top: -8px;
+  right: -5px;
+}
+table.dataTable thead th.ui-state-default {
+  border-right-width: 0;
+}
+table.dataTable thead th.ui-state-default:last-child {
+  border-right-width: 1px;
+}
+table.dataTable tbody tr {
+  background-color: white;
+}
+table.dataTable tbody tr.selected {
+  background-color: #b0bed9;
+}
+table.dataTable tbody th,
+table.dataTable tbody td {
+  padding: 8px 10px;
+}
+table.dataTable th.center,
+table.dataTable td.center,
+table.dataTable td.dataTables_empty {
+  text-align: center;
+}
+table.dataTable th.right,
+table.dataTable td.right {
+  text-align: right;
+}
+table.dataTable.row-border tbody th, table.dataTable.row-border tbody td, table.dataTable.display tbody th, table.dataTable.display tbody td {
+  border-top: 1px solid #dddddd;
+}
+table.dataTable.row-border tbody tr:first-child th,
+table.dataTable.row-border tbody tr:first-child td, table.dataTable.display tbody tr:first-child th,
+table.dataTable.display tbody tr:first-child td {
+  border-top: none;
+}
+table.dataTable.cell-border tbody th, table.dataTable.cell-border tbody td {
+  border-top: 1px solid #dddddd;
+  border-right: 1px solid #dddddd;
+}
+table.dataTable.cell-border tbody tr th:first-child,
+table.dataTable.cell-border tbody tr td:first-child {
+  border-left: 1px solid #dddddd;
+}
+table.dataTable.cell-border tbody tr:first-child th,
+table.dataTable.cell-border tbody tr:first-child td {
+  border-top: none;
+}
+table.dataTable.stripe tbody tr.odd, table.dataTable.display tbody tr.odd {
+  background-color: #f9f9f9;
+}
+table.dataTable.stripe tbody tr.odd.selected, table.dataTable.display tbody tr.odd.selected {
+  background-color: #abb9d3;
+}
+table.dataTable.hover tbody tr:hover,
+table.dataTable.hover tbody tr.odd:hover,
+table.dataTable.hover tbody tr.even:hover, table.dataTable.display tbody tr:hover,
+table.dataTable.display tbody tr.odd:hover,
+table.dataTable.display tbody tr.even:hover {
+  background-color: whitesmoke;
+}
+table.dataTable.hover tbody tr:hover.selected,
+table.dataTable.hover tbody tr.odd:hover.selected,
+table.dataTable.hover tbody tr.even:hover.selected, table.dataTable.display tbody tr:hover.selected,
+table.dataTable.display tbody tr.odd:hover.selected,
+table.dataTable.display tbody tr.even:hover.selected {
+  background-color: #a9b7d1;
+}
+table.dataTable.order-column tbody tr > .sorting_1,
+table.dataTable.order-column tbody tr > .sorting_2,
+table.dataTable.order-column tbody tr > .sorting_3, table.dataTable.display tbody tr > .sorting_1,
+table.dataTable.display tbody tr > .sorting_2,
+table.dataTable.display tbody tr > .sorting_3 {
+  background-color: #f9f9f9;
+}
+table.dataTable.order-column tbody tr.selected > .sorting_1,
+table.dataTable.order-column tbody tr.selected > .sorting_2,
+table.dataTable.order-column tbody tr.selected > .sorting_3, table.dataTable.display tbody tr.selected > .sorting_1,
+table.dataTable.display tbody tr.selected > .sorting_2,
+table.dataTable.display tbody tr.selected > .sorting_3 {
+  background-color: #acbad4;
+}
+table.dataTable.display tbody tr.odd > .sorting_1, table.dataTable.order-column.stripe tbody tr.odd > .sorting_1 {
+  background-color: #f1f1f1;
+}
+table.dataTable.display tbody tr.odd > .sorting_2, table.dataTable.order-column.stripe tbody tr.odd > .sorting_2 {
+  background-color: #f3f3f3;
+}
+table.dataTable.display tbody tr.odd > .sorting_3, table.dataTable.order-column.stripe tbody tr.odd > .sorting_3 {
+  background-color: whitesmoke;
+}
+table.dataTable.display tbody tr.odd.selected > .sorting_1, table.dataTable.order-column.stripe tbody tr.odd.selected > .sorting_1 {
+  background-color: #a6b3cd;
+}
+table.dataTable.display tbody tr.odd.selected > .sorting_2, table.dataTable.order-column.stripe tbody tr.odd.selected > .sorting_2 {
+  background-color: #a7b5ce;
+}
+table.dataTable.display tbody tr.odd.selected > .sorting_3, table.dataTable.order-column.stripe tbody tr.odd.selected > .sorting_3 {
+  background-color: #a9b6d0;
+}
+table.dataTable.display tbody tr.even > .sorting_1, table.dataTable.order-column.stripe tbody tr.even > .sorting_1 {
+  background-color: #f9f9f9;
+}
+table.dataTable.display tbody tr.even > .sorting_2, table.dataTable.order-column.stripe tbody tr.even > .sorting_2 {
+  background-color: #fbfbfb;
+}
+table.dataTable.display tbody tr.even > .sorting_3, table.dataTable.order-column.stripe tbody tr.even > .sorting_3 {
+  background-color: #fdfdfd;
+}
+table.dataTable.display tbody tr.even.selected > .sorting_1, table.dataTable.order-column.stripe tbody tr.even.selected > .sorting_1 {
+  background-color: #acbad4;
+}
+table.dataTable.display tbody tr.even.selected > .sorting_2, table.dataTable.order-column.stripe tbody tr.even.selected > .sorting_2 {
+  background-color: #adbbd6;
+}
+table.dataTable.display tbody tr.even.selected > .sorting_3, table.dataTable.order-column.stripe tbody tr.even.selected > .sorting_3 {
+  background-color: #afbdd8;
+}
+table.dataTable.display tbody tr:hover > .sorting_1,
+table.dataTable.display tbody tr.odd:hover > .sorting_1,
+table.dataTable.display tbody tr.even:hover > .sorting_1, table.dataTable.order-column.hover tbody tr:hover > .sorting_1,
+table.dataTable.order-column.hover tbody tr.odd:hover > .sorting_1,
+table.dataTable.order-column.hover tbody tr.even:hover > .sorting_1 {
+  background-color: #eaeaea;
+}
+table.dataTable.display tbody tr:hover > .sorting_2,
+table.dataTable.display tbody tr.odd:hover > .sorting_2,
+table.dataTable.display tbody tr.even:hover > .sorting_2, table.dataTable.order-column.hover tbody tr:hover > .sorting_2,
+table.dataTable.order-column.hover tbody tr.odd:hover > .sorting_2,
+table.dataTable.order-column.hover tbody tr.even:hover > .sorting_2 {
+  background-color: #ebebeb;
+}
+table.dataTable.display tbody tr:hover > .sorting_3,
+table.dataTable.display tbody tr.odd:hover > .sorting_3,
+table.dataTable.display tbody tr.even:hover > .sorting_3, table.dataTable.order-column.hover tbody tr:hover > .sorting_3,
+table.dataTable.order-column.hover tbody tr.odd:hover > .sorting_3,
+table.dataTable.order-column.hover tbody tr.even:hover > .sorting_3 {
+  background-color: #eeeeee;
+}
+table.dataTable.display tbody tr:hover.selected > .sorting_1,
+table.dataTable.display tbody tr.odd:hover.selected > .sorting_1,
+table.dataTable.display tbody tr.even:hover.selected > .sorting_1, table.dataTable.order-column.hover tbody tr:hover.selected > .sorting_1,
+table.dataTable.order-column.hover tbody tr.odd:hover.selected > .sorting_1,
+table.dataTable.order-column.hover tbody tr.even:hover.selected > .sorting_1 {
+  background-color: #a1aec7;
+}
+table.dataTable.display tbody tr:hover.selected > .sorting_2,
+table.dataTable.display tbody tr.odd:hover.selected > .sorting_2,
+table.dataTable.display tbody tr.even:hover.selected > .sorting_2, table.dataTable.order-column.hover tbody tr:hover.selected > .sorting_2,
+table.dataTable.order-column.hover tbody tr.odd:hover.selected > .sorting_2,
+table.dataTable.order-column.hover tbody tr.even:hover.selected > .sorting_2 {
+  background-color: #a2afc8;
+}
+table.dataTable.display tbody tr:hover.selected > .sorting_3,
+table.dataTable.display tbody tr.odd:hover.selected > .sorting_3,
+table.dataTable.display tbody tr.even:hover.selected > .sorting_3, table.dataTable.order-column.hover tbody tr:hover.selected > .sorting_3,
+table.dataTable.order-column.hover tbody tr.odd:hover.selected > .sorting_3,
+table.dataTable.order-column.hover tbody tr.even:hover.selected > .sorting_3 {
+  background-color: #a4b2cb;
+}
+
+table.dataTable,
+table.dataTable th,
+table.dataTable td {
+  -webkit-box-sizing: content-box;
+  -moz-box-sizing: content-box;
+  box-sizing: content-box;
+}
+
+/*
+ * Control feature layout
+ */
+.dataTables_wrapper {
+  position: relative;
+  clear: both;
+  *zoom: 1;
+  zoom: 1;
+}
+.dataTables_wrapper .dataTables_length {
+  float: left;
+}
+.dataTables_wrapper .dataTables_filter {
+  float: right;
+  text-align: right;
+}
+.dataTables_wrapper .dataTables_filter input {
+  margin-left: 0.5em;
+}
+.dataTables_wrapper .dataTables_info {
+  clear: both;
+  float: left;
+  padding-top: 0.55em;
+}
+.dataTables_wrapper .dataTables_paginate {
+  float: right;
+  text-align: right;
+}
+.dataTables_wrapper .dataTables_paginate .fg-button {
+  box-sizing: border-box;
+  display: inline-block;
+  min-width: 1.5em;
+  padding: 0.5em;
+  margin-left: 2px;
+  text-align: center;
+  text-decoration: none !important;
+  cursor: pointer;
+  *cursor: hand;
+  color: #333333 !important;
+  border: 1px solid transparent;
+}
+.dataTables_wrapper .dataTables_paginate .fg-button:active {
+  outline: none;
+}
+.dataTables_wrapper .dataTables_paginate .fg-button:first-child {
+  border-top-left-radius: 3px;
+  border-bottom-left-radius: 3px;
+}
+.dataTables_wrapper .dataTables_paginate .fg-button:last-child {
+  border-top-right-radius: 3px;
+  border-bottom-right-radius: 3px;
+}
+.dataTables_wrapper .dataTables_processing {
+  position: absolute;
+  top: 50%;
+  left: 50%;
+  width: 100%;
+  height: 40px;
+  margin-left: -50%;
+  margin-top: -25px;
+  padding-top: 20px;
+  text-align: center;
+  font-size: 1.2em;
+  background-color: white;
+  background: -webkit-gradient(linear, left top, right top, color-stop(0%, rgba(255, 255, 255, 0)), color-stop(25%, rgba(255, 255, 255, 0.9)), color-stop(75%, rgba(255, 255, 255, 0.9)), color-stop(100%, rgba(255, 255, 255, 0)));
+  /* Chrome,Safari4+ */
+  background: -webkit-linear-gradient(left, rgba(255, 255, 255, 0) 0%, rgba(255, 255, 255, 0.9) 25%, rgba(255, 255, 255, 0.9) 75%, rgba(255, 255, 255, 0) 100%);
+  /* Chrome10+,Safari5.1+ */
+  background: -moz-linear-gradient(left, rgba(255, 255, 255, 0) 0%, rgba(255, 255, 255, 0.9) 25%, rgba(255, 255, 255, 0.9) 75%, rgba(255, 255, 255, 0) 100%);
+  /* FF3.6+ */
+  background: -ms-linear-gradient(left, rgba(255, 255, 255, 0) 0%, rgba(255, 255, 255, 0.9) 25%, rgba(255, 255, 255, 0.9) 75%, rgba(255, 255, 255, 0) 100%);
+  /* IE10+ */
+  background: -o-linear-gradient(left, rgba(255, 255, 255, 0) 0%, rgba(255, 255, 255, 0.9) 25%, rgba(255, 255, 255, 0.9) 75%, rgba(255, 255, 255, 0) 100%);
+  /* Opera 11.10+ */
+  background: linear-gradient(to right, rgba(255, 255, 255, 0) 0%, rgba(255, 255, 255, 0.9) 25%, rgba(255, 255, 255, 0.9) 75%, rgba(255, 255, 255, 0) 100%);
+  /* W3C */
+}
+.dataTables_wrapper .dataTables_length,
+.dataTables_wrapper .dataTables_filter,
+.dataTables_wrapper .dataTables_info,
+.dataTables_wrapper .dataTables_processing,
+.dataTables_wrapper .dataTables_paginate {
+  color: #333333;
+}
+.dataTables_wrapper .dataTables_scroll {
+  clear: both;
+}
+.dataTables_wrapper .dataTables_scrollBody {
+  *margin-top: -1px;
+  -webkit-overflow-scrolling: touch;
+}
+.dataTables_wrapper .ui-widget-header {
+  font-weight: normal;
+}
+.dataTables_wrapper .ui-toolbar {
+  padding: 8px;
+}
+.dataTables_wrapper:after {
+  visibility: hidden;
+  display: block;
+  content: "";
+  clear: both;
+  height: 0;
+}
+
+@media screen and (max-width: 767px) {
+  .dataTables_wrapper .dataTables_length,
+  .dataTables_wrapper .dataTables_filter,
+  .dataTables_wrapper .dataTables_info,
+  .dataTables_wrapper .dataTables_paginate {
+    float: none;
+    text-align: center;
+  }
+  .dataTables_wrapper .dataTables_filter,
+  .dataTables_wrapper .dataTables_paginate {
+    margin-top: 0.5em;
+  }
+}
+/*
+ * TableTools styling for jQuery UI
+ */
+div.DTTT_container {
+  position: relative;
+  float: left;
+}
+
+.DTTT_button {
+  position: relative;
+  float: left;
+  margin-right: 3px;
+  padding: 3px 10px;
+  border: 1px solid #d0d0d0;
+  background-color: #fff;
+  color: #333 !important;
+  cursor: pointer;
+}
+
+.DTTT_button::-moz-focus-inner {
+  border: none !important;
+  padding: 0;
+}
+
+.DTTT_disabled {
+  color: #999 !important;
+}
+
+table.DTTT_selectable tbody tr {
+  cursor: pointer;
+}
+
+div.DTTT_collection {
+  width: 150px;
+  background-color: #f3f3f3;
+  overflow: hidden;
+  z-index: 2002;
+  box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+  -moz-box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+  -webkit-box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+}
+div.DTTT_collection button.DTTT_button,
+div.DTTT_collection div.DTTT_button,
+div.DTTT_collection a.DTTT_button {
+  float: none;
+  width: 100%;
+  margin-bottom: -0.1em;
+}
+
+div.DTTT_collection_background {
+  background: url(../images/background.png) repeat top left;
+  z-index: 2001;
+}
+
+.DTTT_print_info {
+  position: absolute;
+  top: 50%;
+  left: 50%;
+  width: 400px;
+  height: 150px;
+  margin-left: -200px;
+  margin-top: -75px;
+  text-align: center;
+  background-color: #3f3f3f;
+  color: white;
+  padding: 10px 30px;
+  opacity: 0.9;
+  border-radius: 5px;
+  -moz-border-radius: 5px;
+  -webkit-border-radius: 5px;
+  box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+  -moz-box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+  -webkit-box-shadow: 5px 5px 5px rgba(0, 0, 0, 0.5);
+}
+.DTTT_print_info .DTTT_print_info h6 {
+  font-weight: normal;
+  font-size: 28px;
+  line-height: 28px;
+  margin: 1em;
+}
+.DTTT_print_info .DTTT_print_info p {
+  font-size: 14px;
+  line-height: 20px;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png b/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png
new file mode 100644
index 0000000..86c2baa
Binary files /dev/null and b/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_glass_75_e6e6e6_1x400.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png b/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png
new file mode 100644
index 0000000..7c9fa6c
Binary files /dev/null and b/exec/java-exec/src/main/resources/rest/static/css/images/ui-bg_highlight-soft_75_cccccc_1x100.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/images/ui-icons_888888_256x240.png
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/images/ui-icons_888888_256x240.png b/exec/java-exec/src/main/resources/rest/static/css/images/ui-icons_888888_256x240.png
new file mode 100644
index 0000000..5ba708c
Binary files /dev/null and b/exec/java-exec/src/main/resources/rest/static/css/images/ui-icons_888888_256x240.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/css/jquery-ui-1.10.3.min.css
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/css/jquery-ui-1.10.3.min.css b/exec/java-exec/src/main/resources/rest/static/css/jquery-ui-1.10.3.min.css
new file mode 100644
index 0000000..3bf71df
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/css/jquery-ui-1.10.3.min.css
@@ -0,0 +1,9 @@
+/*! jQuery UI - v1.10.3 - 2013-05-03
+* http://jqueryui.com
+* Includes: jquery.ui.core.css, jquery.ui.accordion.css, jquery.ui.autocomplete.css, jquery.ui.button.css, jquery.ui.datepicker.css, jquery.ui.dialog.css, jquery.ui.menu.css, jquery.ui.progressbar.css, jquery.ui.resizable.css, jquery.ui.selectable.css, jquery.ui.slider.css, jquery.ui.spinner.css, jquery.ui.tabs.css, jquery.ui.tooltip.css
+* To view and modify this theme, visit http://jqueryui.com/themeroller/?ffDefault=Verdana%2CArial%2Csans-serif&fwDefault=normal&fsDefault=1.1em&cornerRadius=4px&bgColorHeader=cccccc&bgTextureHeader=highlight_soft&bgImgOpacityHeader=75&borderColorHeader=aaaaaa&fcHeader=222222&iconColorHeader=222222&bgColorContent=ffffff&bgTextureContent=flat&bgImgOpacityContent=75&borderColorContent=aaaaaa&fcContent=222222&iconColorContent=222222&bgColorDefault=e6e6e6&bgTextureDefault=glass&bgImgOpacityDefault=75&borderColorDefault=d3d3d3&fcDefault=555555&iconColorDefault=888888&bgColorHover=dadada&bgTextureHover=glass&bgImgOpacityHover=75&borderColorHover=999999&fcHover=212121&iconColorHover=454545&bgColorActive=ffffff&bgTextureActive=glass&bgImgOpacityActive=65&borderColorActive=aaaaaa&fcActive=212121&iconColorActive=454545&bgColorHighlight=fbf9ee&bgTextureHighlight=glass&bgImgOpacityHighlight=55&borderColorHighlight=fcefa1&fcHighlight=363636&iconColorHighlight=2e83ff&bgColorError=fef1ec&bgTextureE
 rror=glass&bgImgOpacityError=95&borderColorError=cd0a0a&fcError=cd0a0a&iconColorError=cd0a0a&bgColorOverlay=aaaaaa&bgTextureOverlay=flat&bgImgOpacityOverlay=0&opacityOverlay=30&bgColorShadow=aaaaaa&bgTextureShadow=flat&bgImgOpacityShadow=0&opacityShadow=30&thicknessShadow=8px&offsetTopShadow=-8px&offsetLeftShadow=-8px&cornerRadiusShadow=8px
+* Copyright 2013 jQuery Foundation and other contributors Licensed MIT */
+.ui-helper-hidden{display:none}.ui-helper-hidden-accessible{border:0;clip:rect(0 0 0 0);height:1px;margin:-1px;overflow:hidden;padding:0;position:absolute;width:1px}.ui-helper-reset{margin:0;padding:0;border:0;outline:0;line-height:1.3;text-decoration:none;font-size:100%;list-style:none}.ui-helper-clearfix:before,.ui-helper-clearfix:after{content:"";display:table;border-collapse:collapse}.ui-helper-clearfix:after{clear:both}.ui-helper-clearfix{min-height:0}
+.ui-helper-zfix{width:100%;height:100%;top:0;left:0;position:absolute;opacity:0;filter:Alpha(Opacity=0)}.ui-front{z-index:100}.ui-state-disabled{cursor:default!important}.ui-icon{display:block;text-indent:-99999px;overflow:hidden;background-repeat:no-repeat}.ui-widget-overlay{position:fixed;top:0;left:0;width:100%;height:100%}.ui-accordion .ui-accordion-header{display:block;cursor:pointer;position:relative;margin-top:2px;padding:.5em .5em .5em .7em;min-height:0}.ui-accordion .ui-accordion-icons{padding-left:2.2em}.ui-accordion .ui-accordion-noicons{padding-left:.7em}.ui-accordion .ui-accordion-icons .ui-accordion-icons{padding-left:2.2em}.ui-accordion .ui-accordion-header .ui-accordion-header-icon{position:absolute;left:.5em;top:50%;margin-top:-8px}.ui-accordion .ui-accordion-content{padding:1em 2.2em;border-top:0;overflow:auto}
+.ui-autocomplete{position:absolute;top:0;left:0;cursor:default}.ui-button{display:inline-block;position:relative;padding:0;line-height:normal;margin-right:.1em;cursor:pointer;vertical-align:middle;text-align:center;overflow:visible}.ui-button,.ui-button:link,.ui-button:visited,.ui-button:hover,.ui-button:active{text-decoration:none}.ui-button-icon-only{width:2.2em}button.ui-button-icon-only{width:2.4em}.ui-button-icons-only{width:3.4em}button.ui-button-icons-only{width:3.7em}.ui-button .ui-button-text{display:block;line-height:normal}.ui-button-text-only .ui-button-text{padding:.4em 1em}.ui-button-icon-only
+.ui-button-text,.ui-button-icons-only .ui-button-text{padding:.4em;text-indent:-9999999px}.ui-button-text-icon-primary .ui-button-text,.ui-button-text-icons .ui-button-text{padding:.4em 1em .4em 2.1em}.ui-button-text-icon-secondary .ui-button-text,.ui-button-text-icons .ui-button-text{padding:.4em 2.1em .4em 1em}.ui-button-text-icons .ui-button-text{padding-left:2.1em;padding-right:2.1em}input.ui-button{padding:.4em 1em}.ui-button-icon-only .ui-icon,.ui-button-text-icon-primary .ui-icon,.ui-button-text-icon-secondary .ui-icon,.ui-button-text-icons .ui-icon,.ui-button-icons-only .ui-icon{position:absolute;top:50%;margin-top:-8px}.ui-button-icon-only .ui-icon{left:50%;margin-left:-8px}.ui-button-text-icon-primary .ui-button-icon-primary,.ui-button-text-icons .ui-button-icon-primary,.ui-button-icons-only .ui-button-icon-primary{left:.5em}.ui-button-text-icon-secondary .ui-button-icon-secondary,.ui-button-text-icons .ui-button-icon-secondary,.ui-button-icons-only .ui-button-icon-seconda
 ry{right:.5em}.ui-buttonset{margin-right:7px}.ui-buttonset .ui-button{margin-left:0;margin-right:-.3em}input.ui-button::-moz-focus-inner,button.ui-button::-moz-focus-inner{border:0;padding:0}.ui-datepicker{width:17em;padding:.2em .2em 0;display:none}.ui-datepicker .ui-datepicker-header{position:relative;padding:.2em 0}.ui-datepicker .ui-datepicker-prev,.ui-datepicker .ui-datepicker-next{position:absolute;top:2px;width:1.8em;height:1.8em}.ui-datepicker .ui-datepicker-prev-hover,.ui-datepicker .ui-datepicker-next-hover{top:1px}.ui-datepicker .ui-datepicker-prev{left:2px}.ui-datepicker .ui-datepicker-next{right:2px}.ui-datepicker .ui-datepicker-prev-hover{left:1px}.ui-datepicker .ui-datepicker-next-hover{right:1px}.ui-datepicker .ui-datepicker-prev span,.ui-datepicker .ui-datepicker-next span{display:block;position:absolute;left:50%;margin-left:-8px;top:50%;margin-top:-8px}.ui-datepicker .ui-datepicker-title{margin:0 2.3em;line-height:1.8em;text-align:center}.ui-datepicker .ui-datepick
 er-title select{font-size:1em;margin:1px 0}.ui-datepicker select.ui-datepicker-month-year{width:100%}.ui-datepicker select.ui-datepicker-month,.ui-datepicker select.ui-datepicker-year{width:49%}.ui-datepicker table{width:100%;font-size:.9em;border-collapse:collapse;margin:0 0 .4em}.ui-datepicker th{padding:.7em .3em;text-align:center;font-weight:700;border:0}.ui-datepicker td{border:0;padding:1px}.ui-datepicker td span,.ui-datepicker td a{display:block;padding:.2em;text-align:right;text-decoration:none}.ui-datepicker .ui-datepicker-buttonpane{background-image:none;margin:.7em 0 0;padding:0 .2em;border-left:0;border-right:0;border-bottom:0}.ui-datepicker .ui-datepicker-buttonpane button{float:right;margin:.5em .2em .4em;cursor:pointer;padding:.2em .6em .3em;width:auto;overflow:visible}.ui-datepicker .ui-datepicker-buttonpane button.ui-datepicker-current{float:left}.ui-datepicker.ui-datepicker-multi{width:auto}.ui-datepicker-multi .ui-datepicker-group{float:left}.ui-datepicker-multi .
 ui-datepicker-group table{width:95%;margin:0 auto .4em}.ui-datepicker-multi-2 .ui-datepicker-group{width:50%}.ui-datepicker-multi-3 .ui-datepicker-group{width:33.3%}.ui-datepicker-multi-4 .ui-datepicker-group{width:25%}.ui-datepicker-multi .ui-datepicker-group-last .ui-datepicker-header,.ui-datepicker-multi .ui-datepicker-group-middle .ui-datepicker-header{border-left-width:0}.ui-datepicker-multi .ui-datepicker-buttonpane{clear:left}.ui-datepicker-row-break{clear:both;width:100%;font-size:0}.ui-datepicker-rtl{direction:rtl}.ui-datepicker-rtl .ui-datepicker-prev{right:2px;left:auto}.ui-datepicker-rtl .ui-datepicker-next{left:2px;right:auto}.ui-datepicker-rtl .ui-datepicker-prev:hover{right:1px;left:auto}.ui-datepicker-rtl .ui-datepicker-next:hover{left:1px;right:auto}.ui-datepicker-rtl .ui-datepicker-buttonpane{clear:right}.ui-datepicker-rtl .ui-datepicker-buttonpane button{float:left}.ui-datepicker-rtl .ui-datepicker-buttonpane button.ui-datepicker-current,.ui-datepicker-rtl .ui-dat
 epicker-group{float:right}.ui-datepicker-rtl .ui-datepicker-group-last .ui-datepicker-header,.ui-datepicker-rtl .ui-datepicker-group-middle .ui-datepicker-header{border-right-width:0;border-left-width:1px}.ui-dialog{position:absolute;top:0;left:0;padding:.2em;outline:0}.ui-dialog .ui-dialog-titlebar{padding:.4em 1em;position:relative}.ui-dialog .ui-dialog-title{float:left;margin:.1em 0;white-space:nowrap;width:90%;overflow:hidden;text-overflow:ellipsis}.ui-dialog .ui-dialog-titlebar-close{position:absolute;right:.3em;top:50%;width:21px;margin:-10px 0 0 0;padding:1px;height:20px}.ui-dialog .ui-dialog-content{position:relative;border:0;padding:.5em 1em;background:0;overflow:auto}.ui-dialog .ui-dialog-buttonpane{text-align:left;border-width:1px 0 0;background-image:none;margin-top:.5em;padding:.3em 1em .5em .4em}.ui-dialog .ui-dialog-buttonpane .ui-dialog-buttonset{float:right}.ui-dialog .ui-dialog-buttonpane button{margin:.5em .4em .5em 0;cursor:pointer}.ui-dialog .ui-resizable-se{wid
 th:12px;height:12px;right:-5px;bottom:-5px;background-position:16px 16px}.ui-draggable .ui-dialog-titlebar{cursor:move}.ui-menu{list-style:none;padding:2px;margin:0;display:block;outline:0}.ui-menu .ui-menu{margin-top:-3px;position:absolute}.ui-menu .ui-menu-item{margin:0;padding:0;width:100%;list-style-image:url(data:image/gif;base64,R0lGODlhAQABAIAAAAAAAP///yH5BAEAAAAALAAAAAABAAEAAAIBRAA7)}.ui-menu .ui-menu-divider{margin:5px -2px 5px -2px;height:0;font-size:0;line-height:0;border-width:1px 0 0}.ui-menu .ui-menu-item a{text-decoration:none;display:block;padding:2px .4em;line-height:1.5;min-height:0;font-weight:400}.ui-menu .ui-menu-item a.ui-state-focus,.ui-menu .ui-menu-item a.ui-state-active{font-weight:400;margin:-1px}.ui-menu .ui-state-disabled{font-weight:400;margin:.4em 0 .2em;line-height:1.5}.ui-menu .ui-state-disabled a{cursor:default}.ui-menu-icons{position:relative}.ui-menu-icons .ui-menu-item a{position:relative;padding-left:2em}.ui-menu .ui-icon{position:absolute;top:.
 2em;left:.2em}.ui-menu .ui-menu-icon{position:static;float:right}.ui-progressbar{height:2em;text-align:left;overflow:hidden}.ui-progressbar .ui-progressbar-value{margin:-1px;height:100%}.ui-progressbar .ui-progressbar-overlay{background:url(images/animated-overlay.gif);height:100%;filter:alpha(opacity=25);opacity:.25}.ui-progressbar-indeterminate .ui-progressbar-value{background-image:none}.ui-resizable{position:relative}.ui-resizable-handle{position:absolute;font-size:.1px;display:block}.ui-resizable-disabled .ui-resizable-handle,.ui-resizable-autohide .ui-resizable-handle{display:none}.ui-resizable-n{cursor:n-resize;height:7px;width:100%;top:-5px;left:0}.ui-resizable-s{cursor:s-resize;height:7px;width:100%;bottom:-5px;left:0}.ui-resizable-e{cursor:e-resize;width:7px;right:-5px;top:0;height:100%}.ui-resizable-w{cursor:w-resize;width:7px;left:-5px;top:0;height:100%}.ui-resizable-se{cursor:se-resize;width:12px;height:12px;right:1px;bottom:1px}.ui-resizable-sw{cursor:sw-resize;width:9
 px;height:9px;left:-5px;bottom:-5px}.ui-resizable-nw{cursor:nw-resize;width:9px;height:9px;left:-5px;top:-5px}.ui-resizable-ne{cursor:ne-resize;width:9px;height:9px;right:-5px;top:-5px}.ui-selectable-helper{position:absolute;z-index:100;border:1px dotted #000}.ui-slider{position:relative;text-align:left}.ui-slider .ui-slider-handle{position:absolute;z-index:2;width:1.2em;height:1.2em;cursor:default}.ui-slider .ui-slider-range{position:absolute;z-index:1;font-size:.7em;display:block;border:0;background-position:0 0}.ui-slider.ui-state-disabled .ui-slider-handle,.ui-slider.ui-state-disabled .ui-slider-range{filter:inherit}.ui-slider-horizontal{height:.8em}.ui-slider-horizontal .ui-slider-handle{top:-.3em;margin-left:-.6em}.ui-slider-horizontal .ui-slider-range{top:0;height:100%}.ui-slider-horizontal .ui-slider-range-min{left:0}.ui-slider-horizontal .ui-slider-range-max{right:0}.ui-slider-vertical{width:.8em;height:100px}.ui-slider-vertical .ui-slider-handle{left:-.3em;margin-left:0;ma
 rgin-bottom:-.6em}.ui-slider-vertical .ui-slider-range{left:0;width:100%}.ui-slider-vertical .ui-slider-range-min{bottom:0}.ui-slider-vertical .ui-slider-range-max{top:0}.ui-spinner{position:relative;display:inline-block;overflow:hidden;padding:0;vertical-align:middle}.ui-spinner-input{border:0;background:0;color:inherit;padding:0;margin:.2em 0;vertical-align:middle;margin-left:.4em;margin-right:22px}.ui-spinner-button{width:16px;height:50%;font-size:.5em;padding:0;margin:0;text-align:center;position:absolute;cursor:default;display:block;overflow:hidden;right:0}.ui-spinner a.ui-spinner-button{border-top:0;border-bottom:0;border-right:0}.ui-spinner .ui-icon{position:absolute;margin-top:-8px;top:50%;left:0}.ui-spinner-up{top:0}.ui-spinner-down{bottom:0}.ui-spinner .ui-icon-triangle-1-s{background-position:-65px -16px}.ui-tabs{position:relative;padding:.2em}.ui-tabs .ui-tabs-nav{margin:0;padding:.2em .2em 0}.ui-tabs .ui-tabs-nav li{list-style:none;float:left;position:relative;top:0;mar
 gin:1px .2em 0 0;border-bottom-width:0;padding:0;white-space:nowrap}.ui-tabs .ui-tabs-nav li a{float:left;padding:.5em 1em;text-decoration:none}.ui-tabs .ui-tabs-nav li.ui-tabs-active{margin-bottom:-1px;padding-bottom:1px}.ui-tabs .ui-tabs-nav li.ui-tabs-active a,.ui-tabs .ui-tabs-nav li.ui-state-disabled a,.ui-tabs .ui-tabs-nav li.ui-tabs-loading a{cursor:text}.ui-tabs .ui-tabs-nav li a,.ui-tabs-collapsible .ui-tabs-nav li.ui-tabs-active a{cursor:pointer}.ui-tabs .ui-tabs-panel{display:block;border-width:0;padding:1em 1.4em;background:0}.ui-tooltip{padding:8px;position:absolute;z-index:9999;max-width:300px;-webkit-box-shadow:0 0 5px #aaa;box-shadow:0 0 5px #aaa}body .ui-tooltip{border-width:2px}.ui-widget{font-family:Verdana,Arial,sans-serif;font-size:1.1em}.ui-widget .ui-widget{font-size:1em}.ui-widget input,.ui-widget select,.ui-widget textarea,.ui-widget button{font-family:Verdana,Arial,sans-serif;font-size:1em}.ui-widget-content{border:1px solid #aaa;background:#fff url(images/
 ui-bg_flat_75_ffffff_40x100.png) 50% 50% repeat-x;color:#222}.ui-widget-content a{color:#222}.ui-widget-header{border:1px solid #aaa;background:#ccc url(images/ui-bg_highlight-soft_75_cccccc_1x100.png) 50% 50% repeat-x;color:#222;font-weight:bold}.ui-widget-header a{color:#222}.ui-state-default,.ui-widget-content .ui-state-default,.ui-widget-header .ui-state-default{border:1px solid #d3d3d3;background:#e6e6e6 url(images/ui-bg_glass_75_e6e6e6_1x400.png) 50% 50% repeat-x;font-weight:normal;color:#555}.ui-state-default a,.ui-state-default a:link,.ui-state-default a:visited{color:#555;text-decoration:none}.ui-state-hover,.ui-widget-content .ui-state-hover,.ui-widget-header .ui-state-hover,.ui-state-focus,.ui-widget-content .ui-state-focus,.ui-widget-header .ui-state-focus{border:1px solid #999;background:#dadada url(images/ui-bg_glass_75_dadada_1x400.png) 50% 50% repeat-x;font-weight:normal;color:#212121}.ui-state-hover a,.ui-state-hover a:hover,.ui-state-hover a:link,.ui-state-hover a:
 visited{color:#212121;text-decoration:none}.ui-state-active,.ui-widget-content .ui-state-active,.ui-widget-header .ui-state-active{border:1px solid #aaa;background:#fff url(images/ui-bg_glass_65_ffffff_1x400.png) 50% 50% repeat-x;font-weight:normal;color:#212121}.ui-state-active a,.ui-state-active a:link,.ui-state-active a:visited{color:#212121;text-decoration:none}.ui-state-highlight,.ui-widget-content .ui-state-highlight,.ui-widget-header .ui-state-highlight{border:1px solid #fcefa1;background:#fbf9ee url(images/ui-bg_glass_55_fbf9ee_1x400.png) 50% 50% repeat-x;color:#363636}.ui-state-highlight a,.ui-widget-content .ui-state-highlight a,.ui-widget-header .ui-state-highlight a{color:#363636}.ui-state-error,.ui-widget-content .ui-state-error,.ui-widget-header .ui-state-error{border:1px solid #cd0a0a;background:#fef1ec url(images/ui-bg_glass_95_fef1ec_1x400.png) 50% 50% repeat-x;color:#cd0a0a}.ui-state-error a,.ui-widget-content .ui-state-error a,.ui-widget-header .ui-state-error a{c
 olor:#cd0a0a}.ui-state-error-text,.ui-widget-content .ui-state-error-text,.ui-widget-header .ui-state-error-text{color:#cd0a0a}.ui-priority-primary,.ui-widget-content .ui-priority-primary,.ui-widget-header .ui-priority-primary{font-weight:bold}.ui-priority-secondary,.ui-widget-content .ui-priority-secondary,.ui-widget-header .ui-priority-secondary{opacity:.7;filter:Alpha(Opacity=70);font-weight:normal}.ui-state-disabled,.ui-widget-content .ui-state-disabled,.ui-widget-header .ui-state-disabled{opacity:.35;filter:Alpha(Opacity=35);background-image:none}.ui-state-disabled .ui-icon{filter:Alpha(Opacity=35)}.ui-icon{width:16px;height:16px}.ui-icon,.ui-widget-content .ui-icon{background-image:url(images/ui-icons_222222_256x240.png)}.ui-widget-header .ui-icon{background-image:url(images/ui-icons_222222_256x240.png)}.ui-state-default .ui-icon{background-image:url(images/ui-icons_888888_256x240.png)}.ui-state-hover .ui-icon,.ui-state-focus .ui-icon{background-image:url(images/ui-icons_45454
 5_256x240.png)}.ui-state-active .ui-icon{background-image:url(images/ui-icons_454545_256x240.png)}.ui-state-highlight .ui-icon{background-image:url(images/ui-icons_2e83ff_256x240.png)}.ui-state-error .ui-icon,.ui-state-error-text .ui-icon{background-image:url(images/ui-icons_cd0a0a_256x240.png)}.ui-icon-blank{background-position:16px 16px}.ui-icon-carat-1-n{background-position:0 0}.ui-icon-carat-1-ne{background-position:-16px 0}.ui-icon-carat-1-e{background-position:-32px 0}.ui-icon-carat-1-se{background-position:-48px 0}.ui-icon-carat-1-s{background-position:-64px 0}.ui-icon-carat-1-sw{background-position:-80px 0}.ui-icon-carat-1-w{background-position:-96px 0}.ui-icon-carat-1-nw{background-position:-112px 0}.ui-icon-carat-2-n-s{background-position:-128px 0}.ui-icon-carat-2-e-w{background-position:-144px 0}.ui-icon-triangle-1-n{background-position:0 -16px}.ui-icon-triangle-1-ne{background-position:-16px -16px}.ui-icon-triangle-1-e{background-position:-32px -16px}.ui-icon-triangle-1-
 se{background-position:-48px -16px}.ui-icon-triangle-1-s{background-position:-64px -16px}.ui-icon-triangle-1-sw{background-position:-80px -16px}.ui-icon-triangle-1-w{background-position:-96px -16px}.ui-icon-triangle-1-nw{background-position:-112px -16px}.ui-icon-triangle-2-n-s{background-position:-128px -16px}.ui-icon-triangle-2-e-w{background-position:-144px -16px}.ui-icon-arrow-1-n{background-position:0 -32px}.ui-icon-arrow-1-ne{background-position:-16px -32px}.ui-icon-arrow-1-e{background-position:-32px -32px}.ui-icon-arrow-1-se{background-position:-48px -32px}.ui-icon-arrow-1-s{background-position:-64px -32px}.ui-icon-arrow-1-sw{background-position:-80px -32px}.ui-icon-arrow-1-w{background-position:-96px -32px}.ui-icon-arrow-1-nw{background-position:-112px -32px}.ui-icon-arrow-2-n-s{background-position:-128px -32px}.ui-icon-arrow-2-ne-sw{background-position:-144px -32px}.ui-icon-arrow-2-e-w{background-position:-160px -32px}.ui-icon-arrow-2-se-nw{background-position:-176px -32px}
 .ui-icon-arrowstop-1-n{background-position:-192px -32px}.ui-icon-arrowstop-1-e{background-position:-208px -32px}.ui-icon-arrowstop-1-s{background-position:-224px -32px}.ui-icon-arrowstop-1-w{background-position:-240px -32px}.ui-icon-arrowthick-1-n{background-position:0 -48px}.ui-icon-arrowthick-1-ne{background-position:-16px -48px}.ui-icon-arrowthick-1-e{background-position:-32px -48px}.ui-icon-arrowthick-1-se{background-position:-48px -48px}.ui-icon-arrowthick-1-s{background-position:-64px -48px}.ui-icon-arrowthick-1-sw{background-position:-80px -48px}.ui-icon-arrowthick-1-w{background-position:-96px -48px}.ui-icon-arrowthick-1-nw{background-position:-112px -48px}.ui-icon-arrowthick-2-n-s{background-position:-128px -48px}.ui-icon-arrowthick-2-ne-sw{background-position:-144px -48px}.ui-icon-arrowthick-2-e-w{background-position:-160px -48px}.ui-icon-arrowthick-2-se-nw{background-position:-176px -48px}.ui-icon-arrowthickstop-1-n{background-position:-192px -48px}.ui-icon-arrowthickstop
 -1-e{background-position:-208px -48px}.ui-icon-arrowthickstop-1-s{background-position:-224px -48px}.ui-icon-arrowthickstop-1-w{background-position:-240px -48px}.ui-icon-arrowreturnthick-1-w{background-position:0 -64px}.ui-icon-arrowreturnthick-1-n{background-position:-16px -64px}.ui-icon-arrowreturnthick-1-e{background-position:-32px -64px}.ui-icon-arrowreturnthick-1-s{background-position:-48px -64px}.ui-icon-arrowreturn-1-w{background-position:-64px -64px}.ui-icon-arrowreturn-1-n{background-position:-80px -64px}.ui-icon-arrowreturn-1-e{background-position:-96px -64px}.ui-icon-arrowreturn-1-s{background-position:-112px -64px}.ui-icon-arrowrefresh-1-w{background-position:-128px -64px}.ui-icon-arrowrefresh-1-n{background-position:-144px -64px}.ui-icon-arrowrefresh-1-e{background-position:-160px -64px}.ui-icon-arrowrefresh-1-s{background-position:-176px -64px}.ui-icon-arrow-4{background-position:0 -80px}.ui-icon-arrow-4-diag{background-position:-16px -80px}.ui-icon-extlink{background-p
 osition:-32px -80px}.ui-icon-newwin{background-position:-48px -80px}.ui-icon-refresh{background-position:-64px -80px}.ui-icon-shuffle{background-position:-80px -80px}.ui-icon-transfer-e-w{background-position:-96px -80px}.ui-icon-transferthick-e-w{background-position:-112px -80px}.ui-icon-folder-collapsed{background-position:0 -96px}.ui-icon-folder-open{background-position:-16px -96px}.ui-icon-document{background-position:-32px -96px}.ui-icon-document-b{background-position:-48px -96px}.ui-icon-note{background-position:-64px -96px}.ui-icon-mail-closed{background-position:-80px -96px}.ui-icon-mail-open{background-position:-96px -96px}.ui-icon-suitcase{background-position:-112px -96px}.ui-icon-comment{background-position:-128px -96px}.ui-icon-person{background-position:-144px -96px}.ui-icon-print{background-position:-160px -96px}.ui-icon-trash{background-position:-176px -96px}.ui-icon-locked{background-position:-192px -96px}.ui-icon-unlocked{background-position:-208px -96px}.ui-icon-boo
 kmark{background-position:-224px -96px}.ui-icon-tag{background-position:-240px -96px}.ui-icon-home{background-position:0 -112px}.ui-icon-flag{background-position:-16px -112px}.ui-icon-calendar{background-position:-32px -112px}.ui-icon-cart{background-position:-48px -112px}.ui-icon-pencil{background-position:-64px -112px}.ui-icon-clock{background-position:-80px -112px}.ui-icon-disk{background-position:-96px -112px}.ui-icon-calculator{background-position:-112px -112px}.ui-icon-zoomin{background-position:-128px -112px}.ui-icon-zoomout{background-position:-144px -112px}.ui-icon-search{background-position:-160px -112px}.ui-icon-wrench{background-position:-176px -112px}.ui-icon-gear{background-position:-192px -112px}.ui-icon-heart{background-position:-208px -112px}.ui-icon-star{background-position:-224px -112px}.ui-icon-link{background-position:-240px -112px}.ui-icon-cancel{background-position:0 -128px}.ui-icon-plus{background-position:-16px -128px}.ui-icon-plusthick{background-position:-
 32px -128px}.ui-icon-minus{background-position:-48px -128px}.ui-icon-minusthick{background-position:-64px -128px}.ui-icon-close{background-position:-80px -128px}.ui-icon-closethick{background-position:-96px -128px}.ui-icon-key{background-position:-112px -128px}.ui-icon-lightbulb{background-position:-128px -128px}.ui-icon-scissors{background-position:-144px -128px}.ui-icon-clipboard{background-position:-160px -128px}.ui-icon-copy{background-position:-176px -128px}.ui-icon-contact{background-position:-192px -128px}.ui-icon-image{background-position:-208px -128px}.ui-icon-video{background-position:-224px -128px}.ui-icon-script{background-position:-240px -128px}.ui-icon-alert{background-position:0 -144px}.ui-icon-info{background-position:-16px -144px}.ui-icon-notice{background-position:-32px -144px}.ui-icon-help{background-position:-48px -144px}.ui-icon-check{background-position:-64px -144px}.ui-icon-bullet{background-position:-80px -144px}.ui-icon-radio-on{background-position:-96px -14
 4px}.ui-icon-radio-off{background-position:-112px -144px}.ui-icon-pin-w{background-position:-128px -144px}.ui-icon-pin-s{background-position:-144px -144px}.ui-icon-play{background-position:0 -160px}.ui-icon-pause{background-position:-16px -160px}.ui-icon-seek-next{background-position:-32px -160px}.ui-icon-seek-prev{background-position:-48px -160px}.ui-icon-seek-end{background-position:-64px -160px}.ui-icon-seek-start{background-position:-80px -160px}.ui-icon-seek-first{background-position:-80px -160px}.ui-icon-stop{background-position:-96px -160px}.ui-icon-eject{background-position:-112px -160px}.ui-icon-volume-off{background-position:-128px -160px}.ui-icon-volume-on{background-position:-144px -160px}.ui-icon-power{background-position:0 -176px}.ui-icon-signal-diag{background-position:-16px -176px}.ui-icon-signal{background-position:-32px -176px}.ui-icon-battery-0{background-position:-48px -176px}.ui-icon-battery-1{background-position:-64px -176px}.ui-icon-battery-2{background-positi
 on:-80px -176px}.ui-icon-battery-3{background-position:-96px -176px}.ui-icon-circle-plus{background-position:0 -192px}.ui-icon-circle-minus{background-position:-16px -192px}.ui-icon-circle-close{background-position:-32px -192px}.ui-icon-circle-triangle-e{background-position:-48px -192px}.ui-icon-circle-triangle-s{background-position:-64px -192px}.ui-icon-circle-triangle-w{background-position:-80px -192px}.ui-icon-circle-triangle-n{background-position:-96px -192px}.ui-icon-circle-arrow-e{background-position:-112px -192px}.ui-icon-circle-arrow-s{background-position:-128px -192px}.ui-icon-circle-arrow-w{background-position:-144px -192px}.ui-icon-circle-arrow-n{background-position:-160px -192px}.ui-icon-circle-zoomin{background-position:-176px -192px}.ui-icon-circle-zoomout{background-position:-192px -192px}.ui-icon-circle-check{background-position:-208px -192px}.ui-icon-circlesmall-plus{background-position:0 -208px}.ui-icon-circlesmall-minus{background-position:-16px -208px}.ui-icon-ci
 rclesmall-close{background-position:-32px -208px}.ui-icon-squaresmall-plus{background-position:-48px -208px}.ui-icon-squaresmall-minus{background-position:-64px -208px}.ui-icon-squaresmall-close{background-position:-80px -208px}.ui-icon-grip-dotted-vertical{background-position:0 -224px}.ui-icon-grip-dotted-horizontal{background-position:-16px -224px}.ui-icon-grip-solid-vertical{background-position:-32px -224px}.ui-icon-grip-solid-horizontal{background-position:-48px -224px}.ui-icon-gripsmall-diagonal-se{background-position:-64px -224px}.ui-icon-grip-diagonal-se{background-position:-80px -224px}.ui-corner-all,.ui-corner-top,.ui-corner-left,.ui-corner-tl{border-top-left-radius:4px}.ui-corner-all,.ui-corner-top,.ui-corner-right,.ui-corner-tr{border-top-right-radius:4px}.ui-corner-all,.ui-corner-bottom,.ui-corner-left,.ui-corner-bl{border-bottom-left-radius:4px}.ui-corner-all,.ui-corner-bottom,.ui-corner-right,.ui-corner-br{border-bottom-right-radius:4px}.ui-widget-overlay{background:#a
 aa url(images/ui-bg_flat_0_aaaaaa_40x100.png) 50% 50% repeat-x;opacity:.3;filter:Alpha(Opacity=30)}.ui-widget-shadow{margin:-8px 0 0 -8px;padding:8px;background:#aaa url(images/ui-bg_flat_0_aaaaaa_40x100.png) 50% 50% repeat-x;opacity:.3;filter:Alpha(Opacity=30);border-radius:8px}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/js/dataTables.colVis-1.1.0.min.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/dataTables.colVis-1.1.0.min.js b/exec/java-exec/src/main/resources/rest/static/js/dataTables.colVis-1.1.0.min.js
new file mode 100644
index 0000000..76296c2
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/dataTables.colVis-1.1.0.min.js
@@ -0,0 +1,23 @@
+/*!
+ ColVis 1.1.0
+ ©2010-2014 SpryMedia Ltd - datatables.net/license
+*/
+(function(j,i,k){j=function(d){var e=function(a,b){(!this.CLASS||"ColVis"!=this.CLASS)&&alert("Warning: ColVis must be initialised with the keyword 'new'");"undefined"==typeof b&&(b={});d.fn.dataTable.camelToHungarian&&d.fn.dataTable.camelToHungarian(e.defaults,b);this.s={dt:null,oInit:b,hidden:!0,abOriginal:[]};this.dom={wrapper:null,button:null,collection:null,background:null,catcher:null,buttons:[],groupButtons:[],restore:null};e.aInstances.push(this);this.s.dt=d.fn.dataTable.Api?(new d.fn.dataTable.Api(a)).settings()[0]:
+a;this._fnConstruct(b);return this};e.prototype={button:function(){return this.dom.wrapper},fnRebuild:function(){this.rebuild()},rebuild:function(){for(var a=this.dom.buttons.length-1;0<=a;a--)this.dom.collection.removeChild(this.dom.buttons[a]);this.dom.buttons.splice(0,this.dom.buttons.length);this.dom.restore&&this.dom.restore.parentNode(this.dom.restore);this._fnAddGroups();this._fnAddButtons();this._fnDrawCallback()},_fnConstruct:function(a){this._fnApplyCustomisation(a);var b=this,c,f;this.dom.wrapper=
+i.createElement("div");this.dom.wrapper.className="ColVis";this.dom.button=d("<button />",{"class":!this.s.dt.bJUI?"ColVis_Button ColVis_MasterButton":"ColVis_Button ColVis_MasterButton ui-button ui-state-default"}).append("<span>"+this.s.buttonText+"</span>").bind("mouseover"==this.s.activate?"mouseover":"click",function(a){a.preventDefault();b._fnCollectionShow()}).appendTo(this.dom.wrapper)[0];this.dom.catcher=this._fnDomCatcher();this.dom.collection=this._fnDomCollection();this.dom.background=
+this._fnDomBackground();this._fnAddGroups();this._fnAddButtons();c=0;for(f=this.s.dt.aoColumns.length;c<f;c++)this.s.abOriginal.push(this.s.dt.aoColumns[c].bVisible);this.s.dt.aoDrawCallback.push({fn:function(){b._fnDrawCallback.call(b)},sName:"ColVis"});d(this.s.dt.oInstance).bind("column-reorder",function(a,d,e){c=0;for(f=b.s.aiExclude.length;c<f;c++)b.s.aiExclude[c]=e.aiInvertMapping[b.s.aiExclude[c]];a=b.s.abOriginal.splice(e.iFrom,1)[0];b.s.abOriginal.splice(e.iTo,0,a);b.fnRebuild()});this._fnDrawCallback()},
+_fnApplyCustomisation:function(a){d.extend(!0,this.s,e.defaults,a);!this.s.showAll&&this.s.bShowAll&&(this.s.showAll=this.s.sShowAll);!this.s.restore&&this.s.bRestore&&(this.s.restore=this.s.sRestore);var a=this.s.groups,b=this.s.aoGroups;if(a)for(var c=0,f=a.length;c<f;c++)if(a[c].title&&(b[c].sTitle=a[c].title),a[c].columns)b[c].aiColumns=a[c].columns},_fnDrawCallback:function(){for(var a=this.s.dt.aoColumns,b=this.dom.buttons,c=this.s.aoGroups,f,g=0,h=b.length;g<h;g++)f=b[g],f.__columnIdx!==k&&
+d("input",f).prop("checked",a[f.__columnIdx].bVisible);b=0;for(f=c.length;b<f;b++){a:{for(var g=c[b].aiColumns,h=0,e=g.length;h<e;h++)if(!1===a[g[h]].bVisible){g=!1;break a}g=!0}if(g)d("input",this.dom.groupButtons[b]).prop("checked",!0),d("input",this.dom.groupButtons[b]).prop("indeterminate",!1);else{a:{g=c[b].aiColumns;h=0;for(e=g.length;h<e;h++)if(!0===a[g[h]].bVisible){g=!1;break a}g=!0}g?(d("input",this.dom.groupButtons[b]).prop("checked",!1),d("input",this.dom.groupButtons[b]).prop("indeterminate",
+!1)):d("input",this.dom.groupButtons[b]).prop("indeterminate",!0)}}},_fnAddGroups:function(){var a;if("undefined"!=typeof this.s.aoGroups)for(var b=0,c=this.s.aoGroups.length;b<c;b++)a=this._fnDomGroupButton(b),this.dom.groupButtons.push(a),this.dom.buttons.push(a),this.dom.collection.appendChild(a)},_fnAddButtons:function(){var a,b=this.s.dt.aoColumns;if(-1===d.inArray("all",this.s.aiExclude))for(var c=0,f=b.length;c<f;c++)-1===d.inArray(c,this.s.aiExclude)&&(a=this._fnDomColumnButton(c),a.__columnIdx=
+c,this.dom.buttons.push(a));"alpha"===this.s.order&&this.dom.buttons.sort(function(a,c){var d=b[a.__columnIdx].sTitle,f=b[c.__columnIdx].sTitle;return d===f?0:d<f?-1:1});this.s.restore&&(a=this._fnDomRestoreButton(),a.className+=" ColVis_Restore",this.dom.buttons.push(a));this.s.showAll&&(a=this._fnDomShowAllButton(),a.className+=" ColVis_ShowAll",this.dom.buttons.push(a));d(this.dom.collection).append(this.dom.buttons)},_fnDomRestoreButton:function(){var a=this;return d('<li class="ColVis_Special '+
+(this.s.dt.bJUI?"ui-button ui-state-default":"")+'">'+this.s.restore+"</li>").click(function(){for(var b=0,c=a.s.abOriginal.length;b<c;b++)a.s.dt.oInstance.fnSetColumnVis(b,a.s.abOriginal[b],!1);a._fnAdjustOpenRows();a.s.dt.oInstance.fnAdjustColumnSizing(!1);a.s.dt.oInstance.fnDraw(!1)})[0]},_fnDomShowAllButton:function(){var a=this;return d('<li class="ColVis_Special '+(this.s.dt.bJUI?"ui-button ui-state-default":"")+'">'+this.s.showAll+"</li>").click(function(){for(var b=0,c=a.s.abOriginal.length;b<
+c;b++)-1===a.s.aiExclude.indexOf(b)&&a.s.dt.oInstance.fnSetColumnVis(b,!0,!1);a._fnAdjustOpenRows();a.s.dt.oInstance.fnAdjustColumnSizing(!1);a.s.dt.oInstance.fnDraw(!1)})[0]},_fnDomGroupButton:function(a){var b=this,c=this.s.aoGroups[a];return d('<li class="ColVis_Special '+(this.s.dt.bJUI?"ui-button ui-state-default":"")+'"><label><input type="checkbox" /><span>'+c.sTitle+"</span></label></li>").click(function(a){var g=!d("input",this).is(":checked");"li"!==a.target.nodeName.toLowerCase()&&(g=!g);
+for(a=0;a<c.aiColumns.length;a++)b.s.dt.oInstance.fnSetColumnVis(c.aiColumns[a],g)})[0]},_fnDomColumnButton:function(a){var b=this,c=this.s.dt.aoColumns[a],f=this.s.dt,c=null===this.s.fnLabel?c.sTitle:this.s.fnLabel(a,c.sTitle,c.nTh);return d("<li "+(f.bJUI?'class="ui-button ui-state-default"':"")+'><label><input type="checkbox" /><span>'+c+"</span></label></li>").click(function(c){var e=!d("input",this).is(":checked");"li"!==c.target.nodeName.toLowerCase()&&(e=!e);c=d.fn.dataTableExt.iApiIndex;d.fn.dataTableExt.iApiIndex=
+b._fnDataTablesApiIndex.call(b);f.oFeatures.bServerSide?(b.s.dt.oInstance.fnSetColumnVis(a,e,!1),b.s.dt.oInstance.fnAdjustColumnSizing(!1),(""!==f.oScroll.sX||""!==f.oScroll.sY)&&b.s.dt.oInstance.oApi._fnScrollDraw(b.s.dt),b._fnDrawCallback()):b.s.dt.oInstance.fnSetColumnVis(a,e);d.fn.dataTableExt.iApiIndex=c;null!==b.s.fnStateChange&&b.s.fnStateChange.call(b,a,e)})[0]},_fnDataTablesApiIndex:function(){for(var a=0,b=this.s.dt.oInstance.length;a<b;a++)if(this.s.dt.oInstance[a]==this.s.dt.nTable)return a;
+return 0},_fnDomCollection:function(){return d("<ul />",{"class":!this.s.dt.bJUI?"ColVis_collection":"ColVis_collection ui-buttonset ui-buttonset-multi"}).css({display:"none",opacity:0,position:!this.s.bCssPosition?"absolute":""})[0]},_fnDomCatcher:function(){var a=this,b=i.createElement("div");b.className="ColVis_catcher";d(b).click(function(){a._fnCollectionHide.call(a,null,null)});return b},_fnDomBackground:function(){var a=this,b=d("<div></div>").addClass("ColVis_collectionBackground").css("opacity",
+0).click(function(){a._fnCollectionHide.call(a,null,null)});"mouseover"==this.s.activate&&b.mouseover(function(){a.s.overcollection=!1;a._fnCollectionHide.call(a,null,null)});return b[0]},_fnCollectionShow:function(){var a=this,b;b=d(this.dom.button).offset();var c=this.dom.collection,f=this.dom.background,e=parseInt(b.left,10),h=parseInt(b.top+d(this.dom.button).outerHeight(),10);this.s.bCssPosition||(c.style.top=h+"px",c.style.left=e+"px");d(c).css({display:"block",opacity:0});f.style.bottom="0px";
+f.style.right="0px";h=this.dom.catcher.style;h.height=d(this.dom.button).outerHeight()+"px";h.width=d(this.dom.button).outerWidth()+"px";h.top=b.top+"px";h.left=e+"px";i.body.appendChild(f);i.body.appendChild(c);i.body.appendChild(this.dom.catcher);d(c).animate({opacity:1},a.s.iOverlayFade);d(f).animate({opacity:0.1},a.s.iOverlayFade,"linear",function(){d.browser&&(d.browser.msie&&d.browser.version=="6.0")&&a._fnDrawCallback()});this.s.bCssPosition||(b="left"==this.s.sAlign?e:e-d(c).outerWidth()+
+d(this.dom.button).outerWidth(),c.style.left=b+"px",f=d(c).outerWidth(),d(c).outerHeight(),e=d(i).width(),b+f>e&&(c.style.left=e-f+"px"));this.s.hidden=!1},_fnCollectionHide:function(){var a=this;!this.s.hidden&&null!==this.dom.collection&&(this.s.hidden=!0,d(this.dom.collection).animate({opacity:0},a.s.iOverlayFade,function(){this.style.display="none"}),d(this.dom.background).animate({opacity:0},a.s.iOverlayFade,function(){i.body.removeChild(a.dom.background);i.body.removeChild(a.dom.catcher)}))},
+_fnAdjustOpenRows:function(){for(var a=this.s.dt.aoOpenRows,b=this.s.dt.oApi._fnVisbleColumns(this.s.dt),c=0,d=a.length;c<d;c++)a[c].nTr.getElementsByTagName("td")[0].colSpan=b}};e.fnRebuild=function(a){var b=null;"undefined"!=typeof a&&(b=a.fnSettings().nTable);for(var c=0,d=e.aInstances.length;c<d;c++)("undefined"==typeof a||b==e.aInstances[c].s.dt.nTable)&&e.aInstances[c].fnRebuild()};e.defaults={active:"click",buttonText:"Show / hide columns",aiExclude:[],bRestore:!1,sRestore:"Restore original",
+bShowAll:!1,sShowAll:"Show All",sAlign:"left",fnStateChange:null,iOverlayFade:500,fnLabel:null,bCssPosition:!1,aoGroups:[],order:"column"};e.aInstances=[];e.prototype.CLASS="ColVis";e.VERSION="1.1.0";e.prototype.VERSION=e.VERSION;"function"==typeof d.fn.dataTable&&"function"==typeof d.fn.dataTableExt.fnVersionCheck&&d.fn.dataTableExt.fnVersionCheck("1.7.0")?d.fn.dataTableExt.aoFeatures.push({fnInit:function(a){var b=a.oInit;return(new e(a,b.colVis||b.oColVis||{})).button()},cFeature:"C",sFeature:"ColVis"}):
+alert("Warning: ColVis requires DataTables 1.7 or greater - www.datatables.net/download");d.fn.dataTable.ColVis=e;return d.fn.DataTable.ColVis=e};"function"===typeof define&&define.amd?define("datatables-colvis",["jquery","datatables"],j):jQuery&&!jQuery.fn.dataTable.ColVis&&j(jQuery,jQuery.fn.dataTable)})(window,document);
\ No newline at end of file


[13/13] drill git commit: Drill-5698 Escape version number period separator, this captures version numbers with 4 5 or 6 in them, like 1.8.0_144

Posted by jn...@apache.org.
Drill-5698 Escape version number period separator, this captures version numbers with 4 5 or 6 in them, like 1.8.0_144

release build

build example

remove print statements

close #890


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/109b2c09
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/109b2c09
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/109b2c09

Branch: refs/heads/master
Commit: 109b2c09433789babb1704e30429450806dd5b37
Parents: 1cec10b
Author: Darren Brien <da...@aol.com>
Authored: Mon Jul 31 11:52:41 2017 +0100
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 contrib/native/client/CMakeLists.txt            |  2 +-
 .../native/client/example/querySubmitter.cpp    |  5 +-
 .../client/src/include/drill/drillClient.hpp    | 70 ++++++++++----------
 distribution/src/resources/drill-config.sh      |  2 +-
 4 files changed, 38 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/109b2c09/contrib/native/client/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/CMakeLists.txt b/contrib/native/client/CMakeLists.txt
index 7b54b00..796e118 100644
--- a/contrib/native/client/CMakeLists.txt
+++ b/contrib/native/client/CMakeLists.txt
@@ -93,7 +93,7 @@ else()
     #    of boost. Arbirtarily, we choose the new namspace to be drill_boost.
     #    See the instructions in the readme for linux/macos and rebuild boost. Then
     #    uncomment the line below to build
-    #    set(Boost_NAMESPACE drill_boost)
+        set(Boost_NAMESPACE drill_boost)
 endif()
 
 find_package(Boost 1.53.0 REQUIRED COMPONENTS regex system date_time chrono thread random)

http://git-wip-us.apache.org/repos/asf/drill/blob/109b2c09/contrib/native/client/example/querySubmitter.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/example/querySubmitter.cpp b/contrib/native/client/example/querySubmitter.cpp
index 47e55de..4fdfe17 100644
--- a/contrib/native/client/example/querySubmitter.cpp
+++ b/contrib/native/client/example/querySubmitter.cpp
@@ -84,7 +84,6 @@ Drill::status_t QueryResultsListener(void* ctx, Drill::RecordBatch* b, Drill::Dr
     if(!err){
         if(b!=NULL){
             b->print(std::cout, 0); // print all rows
-            std::cout << "DATA RECEIVED ..." << std::endl;
             delete b; // we're done with this batch, we can delete it
             if(bTestCancel){
                 return Drill::QRY_FAILURE;
@@ -92,7 +91,6 @@ Drill::status_t QueryResultsListener(void* ctx, Drill::RecordBatch* b, Drill::Dr
                 return Drill::QRY_SUCCESS ;
             }
         }else{
-            std::cout << "Query Complete." << std::endl;
             return Drill::QRY_SUCCESS;
 		}
     }else{
@@ -153,7 +151,7 @@ void print(const Drill::FieldMetadata* pFieldMetadata, void* buf, size_t sz){
             sprintf((char*)printBuffer, "NIY");
             break;
     }
-    printf("%s\t", (char*)printBuffer);
+    printf("%s ,", (char*)printBuffer);
     return;
 }
 
@@ -397,7 +395,6 @@ int main(int argc, char* argv[]) {
             std::cerr<< "Failed to connect with error: "<< client.getError() << " (Using:"<<connectStr<<")"<<std::endl;
             return -1;
         }
-        std::cout<< "Connected!\n" << std::endl;
         if(api=="meta") {
         	Drill::Metadata* metadata = client.getMetadata();
         	if (metadata) {

http://git-wip-us.apache.org/repos/asf/drill/blob/109b2c09/contrib/native/client/src/include/drill/drillClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index f09d7f5..326dd3a 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -338,15 +338,15 @@ namespace meta {
     _DL_INTERVAL_MINUTE_TO_SECOND = 1 << 16L
   };
 
-  template inline _DateTimeLiteralSupport operator&(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport operator|(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport operator^(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport operator&(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport operator|(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport operator^(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
 
-  template inline _DateTimeLiteralSupport& operator&=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport& operator|=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
-  template inline _DateTimeLiteralSupport& operator^=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport& operator&=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport& operator|=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template  _DateTimeLiteralSupport& operator^=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
 
-  template inline _DateTimeLiteralSupport operator~(_DateTimeLiteralSupport __a);
+  template  _DateTimeLiteralSupport operator~(_DateTimeLiteralSupport __a);
 
   /**
    * Date time literal support flags
@@ -396,15 +396,15 @@ namespace meta {
       _C_GROUPBY    = 1 << 1L
   };
 
-  template inline _CollateSupport operator&(_CollateSupport __a, _CollateSupport __b);
-  template inline _CollateSupport operator|(_CollateSupport __a, _CollateSupport __b);
-  template inline _CollateSupport operator^(_CollateSupport __a, _CollateSupport __b);
+  template  _CollateSupport operator&(_CollateSupport __a, _CollateSupport __b);
+  template  _CollateSupport operator|(_CollateSupport __a, _CollateSupport __b);
+  template  _CollateSupport operator^(_CollateSupport __a, _CollateSupport __b);
 
-  template inline _CollateSupport& operator&=(_CollateSupport& __a, _CollateSupport __b);
-  template inline _CollateSupport& operator|=(_CollateSupport& __a, _CollateSupport __b);
-  template inline _CollateSupport& operator^=(_CollateSupport& __a, _CollateSupport __b);
+  template  _CollateSupport& operator&=(_CollateSupport& __a, _CollateSupport __b);
+  template  _CollateSupport& operator|=(_CollateSupport& __a, _CollateSupport __b);
+  template  _CollateSupport& operator^=(_CollateSupport& __a, _CollateSupport __b);
 
-  template inline _CollateSupport operator~(_CollateSupport __a);
+  template  _CollateSupport operator~(_CollateSupport __a);
 
 
   /**
@@ -476,15 +476,15 @@ namespace meta {
       _OJ_ALL_COMPARISON_OPS    = 1 << 7L //!< _OJ_ALL_COMPARISON_OPS
   };
 
-  template inline _OuterJoinSupport operator&(_OuterJoinSupport __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport operator|(_OuterJoinSupport __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport operator^(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport operator&(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport operator|(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport operator^(_OuterJoinSupport __a, _OuterJoinSupport __b);
 
-  template inline _OuterJoinSupport& operator&=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport& operator|=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
-  template inline _OuterJoinSupport& operator^=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport& operator&=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport& operator|=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template  _OuterJoinSupport& operator^=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
 
-  template inline _OuterJoinSupport operator~(_OuterJoinSupport __a);
+  template  _OuterJoinSupport operator~(_OuterJoinSupport __a);
 
   /**
    * Outer join support flags
@@ -541,15 +541,15 @@ namespace meta {
       _SQ_IN_QUANTIFIED = 1 << 5L
   };
 
-  template inline _SubQuerySupport operator&(_SubQuerySupport __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport operator|(_SubQuerySupport __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport operator^(_SubQuerySupport __a, _SubQuerySupport __b);
+  template  _SubQuerySupport operator&(_SubQuerySupport __a, _SubQuerySupport __b);
+  template  _SubQuerySupport operator|(_SubQuerySupport __a, _SubQuerySupport __b);
+  template  _SubQuerySupport operator^(_SubQuerySupport __a, _SubQuerySupport __b);
 
-  template inline _SubQuerySupport& operator&=(_SubQuerySupport& __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport& operator|=(_SubQuerySupport& __a, _SubQuerySupport __b);
-  template inline _SubQuerySupport& operator^=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template  _SubQuerySupport& operator&=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template  _SubQuerySupport& operator|=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template  _SubQuerySupport& operator^=(_SubQuerySupport& __a, _SubQuerySupport __b);
 
-  template inline _SubQuerySupport operator~(_SubQuerySupport __a);
+  template  _SubQuerySupport operator~(_SubQuerySupport __a);
 
   /**
    * SubQuery support flags
@@ -579,15 +579,15 @@ namespace meta {
       _U_UNION_ALL  = 1 << 2L //!< _U_UNION_ALL
   };
 
-  template inline _UnionSupport operator&(_UnionSupport __a, _UnionSupport __b);
-  template inline _UnionSupport operator|(_UnionSupport __a, _UnionSupport __b);
-  template inline _UnionSupport operator^(_UnionSupport __a, _UnionSupport __b);
+  template  _UnionSupport operator&(_UnionSupport __a, _UnionSupport __b);
+  template  _UnionSupport operator|(_UnionSupport __a, _UnionSupport __b);
+  template  _UnionSupport operator^(_UnionSupport __a, _UnionSupport __b);
 
-  template inline _UnionSupport& operator&=(_UnionSupport& __a, _UnionSupport __b);
-  template inline _UnionSupport& operator|=(_UnionSupport& __a, _UnionSupport __b);
-  template inline _UnionSupport& operator^=(_UnionSupport& __a, _UnionSupport __b);
+  template  _UnionSupport& operator&=(_UnionSupport& __a, _UnionSupport __b);
+  template  _UnionSupport& operator|=(_UnionSupport& __a, _UnionSupport __b);
+  template  _UnionSupport& operator^=(_UnionSupport& __a, _UnionSupport __b);
 
-  template inline _UnionSupport operator~(_UnionSupport __a);
+  template  _UnionSupport operator~(_UnionSupport __a);
 
   /**
    * Union support flags

http://git-wip-us.apache.org/repos/asf/drill/blob/109b2c09/distribution/src/resources/drill-config.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-config.sh b/distribution/src/resources/drill-config.sh
index 32306ee..92119e5 100644
--- a/distribution/src/resources/drill-config.sh
+++ b/distribution/src/resources/drill-config.sh
@@ -393,7 +393,7 @@ if [ ! -e "$JAVA" ]; then
 fi
 
 # Ensure that Java version is at least 1.7
-"$JAVA" -version 2>&1 | grep "version" | egrep -e "1.4|1.5|1.6" > /dev/null
+"$JAVA" -version 2>&1 | grep "version" | egrep -e "1\.4|1\.5|1\.6" > /dev/null
 if [ $? -eq 0 ]; then
   fatal_error "Java 1.7 or later is required to run Apache Drill."
 fi


[09/13] drill git commit: DRILL-5699: Drill Web UI Page Source Has Links To External Sites

Posted by jn...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/js/jquery.dataTables-1.10.0.min.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/jquery.dataTables-1.10.0.min.js b/exec/java-exec/src/main/resources/rest/static/js/jquery.dataTables-1.10.0.min.js
new file mode 100644
index 0000000..8a495db
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/jquery.dataTables-1.10.0.min.js
@@ -0,0 +1,152 @@
+/*! DataTables 1.10.0
+ * ©2008-2014 SpryMedia Ltd - datatables.net/license
+ */
+(function(za,N,l){var M=function(g){function S(a){var b,c,d={};g.each(a,function(e){if((b=e.match(/^([^A-Z]+?)([A-Z])/))&&-1!=="a aa ai ao as b fn i m o s ".indexOf(b[1]+" "))c=e.replace(b[0],b[2].toLowerCase()),d[c]=e,"o"===b[1]&&S(a[e])});a._hungarianMap=d}function G(a,b,c){a._hungarianMap||S(a);var d;g.each(b,function(e){d=a._hungarianMap[e];if(d!==l&&(c||b[d]===l))"o"===d.charAt(0)?(b[d]||(b[d]={}),g.extend(!0,b[d],b[e]),G(a[d],b[d],c)):b[d]=b[e]})}function M(a){var b=p.defaults.oLanguage,c=a.sZeroRecords;
+!a.sEmptyTable&&(c&&"No data available in table"===b.sEmptyTable)&&D(a,a,"sZeroRecords","sEmptyTable");!a.sLoadingRecords&&(c&&"Loading..."===b.sLoadingRecords)&&D(a,a,"sZeroRecords","sLoadingRecords");a.sInfoThousands&&(a.sThousands=a.sInfoThousands);(a=a.sDecimal)&&bb(a)}function cb(a){w(a,"ordering","bSort");w(a,"orderMulti","bSortMulti");w(a,"orderClasses","bSortClasses");w(a,"orderCellsTop","bSortCellsTop");w(a,"order","aaSorting");w(a,"orderFixed","aaSortingFixed");w(a,"paging","bPaginate");
+w(a,"pagingType","sPaginationType");w(a,"pageLength","iDisplayLength");w(a,"searching","bFilter")}function db(a){w(a,"orderable","bSortable");w(a,"orderData","aDataSort");w(a,"orderSequence","asSorting");w(a,"orderDataType","sortDataType")}function eb(a){var a=a.oBrowser,b=g("<div/>").css({position:"absolute",top:0,left:0,height:1,width:1,overflow:"hidden"}).append(g("<div/>").css({position:"absolute",top:1,left:1,width:100,overflow:"scroll"}).append(g('<div class="test"/>').css({width:"100%",height:10}))).appendTo("body"),
+c=b.find(".test");a.bScrollOversize=100===c[0].offsetWidth;a.bScrollbarLeft=1!==c.offset().left;b.remove()}function fb(a,b,c,d,e,f){var h,i=!1;c!==l&&(h=c,i=!0);for(;d!==e;)a.hasOwnProperty(d)&&(h=i?b(h,a[d],d,a):a[d],i=!0,d+=f);return h}function Aa(a,b){var c=p.defaults.column,d=a.aoColumns.length,c=g.extend({},p.models.oColumn,c,{nTh:b?b:N.createElement("th"),sTitle:c.sTitle?c.sTitle:b?b.innerHTML:"",aDataSort:c.aDataSort?c.aDataSort:[d],mData:c.mData?c.mData:d,idx:d});a.aoColumns.push(c);c=a.aoPreSearchCols;
+c[d]=g.extend({},p.models.oSearch,c[d]);fa(a,d,null)}function fa(a,b,c){var d=a.aoColumns[b],b=a.oClasses,e=g(d.nTh);if(!d.sWidthOrig){d.sWidthOrig=e.attr("width")||null;var f=(e.attr("style")||"").match(/width:\s*(\d+[pxem%])/);f&&(d.sWidthOrig=f[1])}c!==l&&null!==c&&(db(c),G(p.defaults.column,c),c.mDataProp!==l&&!c.mData&&(c.mData=c.mDataProp),c.sType&&(d._sManualType=c.sType),c.className&&!c.sClass&&(c.sClass=c.className),g.extend(d,c),D(d,c,"sWidth","sWidthOrig"),"number"===typeof c.iDataSort&&
+(d.aDataSort=[c.iDataSort]),D(d,c,"aDataSort"));var c=d.mData,h=T(c),i=d.mRender?T(d.mRender):null,f=function(a){return"string"===typeof a&&-1!==a.indexOf("@")};d._bAttrSrc=g.isPlainObject(c)&&(f(c.sort)||f(c.type)||f(c.filter));d.fnGetData=function(a,b){var c=h(a,b);return d.mRender&&b&&""!==b?i(c,b,a):c};d.fnSetData=Ba(c);a.oFeatures.bSort||(d.bSortable=!1,e.addClass(b.sSortableNone));a=-1!==g.inArray("asc",d.asSorting);e=-1!==g.inArray("desc",d.asSorting);!d.bSortable||!a&&!e?(d.sSortingClass=
+b.sSortableNone,d.sSortingClassJUI=""):a&&!e?(d.sSortingClass=b.sSortableAsc,d.sSortingClassJUI=b.sSortJUIAscAllowed):!a&&e?(d.sSortingClass=b.sSortableDesc,d.sSortingClassJUI=b.sSortJUIDescAllowed):(d.sSortingClass=b.sSortable,d.sSortingClassJUI=b.sSortJUI)}function U(a){if(!1!==a.oFeatures.bAutoWidth){var b=a.aoColumns;Ca(a);for(var c=0,d=b.length;c<d;c++)b[c].nTh.style.width=b[c].sWidth}b=a.oScroll;(""!==b.sY||""!==b.sX)&&V(a);t(a,null,"column-sizing",[a])}function ga(a,b){var c=W(a,"bVisible");
+return"number"===typeof c[b]?c[b]:null}function X(a,b){var c=W(a,"bVisible"),c=g.inArray(b,c);return-1!==c?c:null}function Y(a){return W(a,"bVisible").length}function W(a,b){var c=[];g.map(a.aoColumns,function(a,e){a[b]&&c.push(e)});return c}function Da(a){var b=a.aoColumns,c=a.aoData,d=p.ext.type.detect,e,f,h,i,j,g,m,o,k;e=0;for(f=b.length;e<f;e++)if(m=b[e],k=[],!m.sType&&m._sManualType)m.sType=m._sManualType;else if(!m.sType){h=0;for(i=d.length;h<i;h++){j=0;for(g=c.length;j<g&&!(k[j]===l&&(k[j]=
+A(a,j,e,"type")),o=d[h](k[j],a),!o||"html"===o);j++);if(o){m.sType=o;break}}m.sType||(m.sType="string")}}function gb(a,b,c,d){var e,f,h,i,j,n,m=a.aoColumns;if(b)for(e=b.length-1;0<=e;e--){n=b[e];var o=n.targets!==l?n.targets:n.aTargets;g.isArray(o)||(o=[o]);f=0;for(h=o.length;f<h;f++)if("number"===typeof o[f]&&0<=o[f]){for(;m.length<=o[f];)Aa(a);d(o[f],n)}else if("number"===typeof o[f]&&0>o[f])d(m.length+o[f],n);else if("string"===typeof o[f]){i=0;for(j=m.length;i<j;i++)("_all"==o[f]||g(m[i].nTh).hasClass(o[f]))&&
+d(i,n)}}if(c){e=0;for(a=c.length;e<a;e++)d(e,c[e])}}function H(a,b,c,d){var e=a.aoData.length,f=g.extend(!0,{},p.models.oRow,{src:c?"dom":"data"});f._aData=b;a.aoData.push(f);for(var b=a.aoColumns,f=0,h=b.length;f<h;f++)c&&Ea(a,e,f,A(a,e,f)),b[f].sType=null;a.aiDisplayMaster.push(e);a.oFeatures.bDeferRender||Fa(a,e,c,d);return e}function ha(a,b){var c;b instanceof g||(b=g(b));return b.map(function(b,e){c=ia(a,e);return H(a,c.data,e,c.cells)})}function A(a,b,c,d){var c=a.aoColumns[c],e=a.aoData[b]._aData,
+f=c.fnGetData(e,d);if(f===l)return a.iDrawError!=a.iDraw&&null===c.sDefaultContent&&(O(a,0,"Requested unknown parameter "+("function"==typeof c.mData?"{function}":"'"+c.mData+"'")+" for row "+b,4),a.iDrawError=a.iDraw),c.sDefaultContent;if((f===e||null===f)&&null!==c.sDefaultContent)f=c.sDefaultContent;else if("function"===typeof f)return f();return null===f&&"display"==d?"":f}function Ea(a,b,c,d){a.aoColumns[c].fnSetData(a.aoData[b]._aData,d)}function Ga(a){return g.map(a.match(/(\\.|[^\.])+/g),
+function(a){return a.replace("\\.",".")})}function T(a){if(g.isPlainObject(a)){var b={};g.each(a,function(a,c){c&&(b[a]=T(c))});return function(a,c,f){var h=b[c]||b._;return h!==l?h(a,c,f):a}}if(null===a)return function(a){return a};if("function"===typeof a)return function(b,c,f){return a(b,c,f)};if("string"===typeof a&&(-1!==a.indexOf(".")||-1!==a.indexOf("[")||-1!==a.indexOf("("))){var c=function(a,b,f){var h,i;if(""!==f){i=Ga(f);for(var j=0,g=i.length;j<g;j++){f=i[j].match(Z);h=i[j].match(P);if(f){i[j]=
+i[j].replace(Z,"");""!==i[j]&&(a=a[i[j]]);h=[];i.splice(0,j+1);i=i.join(".");j=0;for(g=a.length;j<g;j++)h.push(c(a[j],b,i));a=f[0].substring(1,f[0].length-1);a=""===a?h:h.join(a);break}else if(h){i[j]=i[j].replace(P,"");a=a[i[j]]();continue}if(null===a||a[i[j]]===l)return l;a=a[i[j]]}}return a};return function(b,e){return c(b,e,a)}}return function(b){return b[a]}}function Ba(a){if(g.isPlainObject(a))return Ba(a._);if(null===a)return function(){};if("function"===typeof a)return function(b,d){a(b,"set",
+d)};if("string"===typeof a&&(-1!==a.indexOf(".")||-1!==a.indexOf("[")||-1!==a.indexOf("("))){var b=function(a,d,e){var e=Ga(e),f;f=e[e.length-1];for(var h,i,j=0,g=e.length-1;j<g;j++){h=e[j].match(Z);i=e[j].match(P);if(h){e[j]=e[j].replace(Z,"");a[e[j]]=[];f=e.slice();f.splice(0,j+1);h=f.join(".");i=0;for(g=d.length;i<g;i++)f={},b(f,d[i],h),a[e[j]].push(f);return}i&&(e[j]=e[j].replace(P,""),a=a[e[j]](d));if(null===a[e[j]]||a[e[j]]===l)a[e[j]]={};a=a[e[j]]}if(f.match(P))a[f.replace(P,"")](d);else a[f.replace(Z,
+"")]=d};return function(c,d){return b(c,d,a)}}return function(b,d){b[a]=d}}function Ha(a){return B(a.aoData,"_aData")}function ja(a){a.aoData.length=0;a.aiDisplayMaster.length=0;a.aiDisplay.length=0}function ka(a,b,c){for(var d=-1,e=0,f=a.length;e<f;e++)a[e]==b?d=e:a[e]>b&&a[e]--; -1!=d&&c===l&&a.splice(d,1)}function la(a,b,c,d){var e=a.aoData[b],f;if("dom"===c||(!c||"auto"===c)&&"dom"===e.src)e._aData=ia(a,e).data;else{var h=e.anCells;if(h){c=0;for(f=h.length;c<f;c++)h[c].innerHTML=A(a,b,c,"display")}}e._aSortData=
+null;e._aFilterData=null;a=a.aoColumns;if(d!==l)a[d].sType=null;else{c=0;for(f=a.length;c<f;c++)a[c].sType=null}Ia(e)}function ia(a,b){var c=[],d=[],e=b.firstChild,f,h,i,j=0,n,m=a.aoColumns,o=function(a,b,c){"string"===typeof a&&(b=a.indexOf("@"),-1!==b&&(a=a.substring(b+1),i["@"+a]=c.getAttribute(a)))},k=function(a){h=m[j];n=g.trim(a.innerHTML);h&&h._bAttrSrc?(i={display:n},o(h.mData.sort,i,a),o(h.mData.type,i,a),o(h.mData.filter,i,a),c.push(i)):c.push(n);d.push(a);j++};if(e)for(;e;)f=e.nodeName.toUpperCase(),
+("TD"==f||"TH"==f)&&k(e),e=e.nextSibling;else{d=b.anCells;e=0;for(f=d.length;e<f;e++)k(d[e])}return{data:c,cells:d}}function Fa(a,b,c,d){var e=a.aoData[b],f=e._aData,h=[],i,j,g,m,o;if(null===e.nTr){i=c||N.createElement("tr");e.nTr=i;e.anCells=h;i._DT_RowIndex=b;Ia(e);m=0;for(o=a.aoColumns.length;m<o;m++){g=a.aoColumns[m];j=c?d[m]:N.createElement(g.sCellType);h.push(j);if(!c||g.mRender||g.mData!==m)j.innerHTML=A(a,b,m,"display");g.sClass&&(j.className+=" "+g.sClass);g.bVisible&&!c?i.appendChild(j):
+!g.bVisible&&c&&j.parentNode.removeChild(j);g.fnCreatedCell&&g.fnCreatedCell.call(a.oInstance,j,A(a,b,m,"display"),f,b,m)}t(a,"aoRowCreatedCallback",null,[i,f,b])}e.nTr.setAttribute("role","row")}function Ia(a){var b=a.nTr,c=a._aData;if(b){c.DT_RowId&&(b.id=c.DT_RowId);if(c.DT_RowClass){var d=c.DT_RowClass.split(" ");a.__rowc=a.__rowc?Ja(a.__rowc.concat(d)):d;g(b).removeClass(a.__rowc.join(" ")).addClass(c.DT_RowClass)}c.DT_RowData&&g(b).data(c.DT_RowData)}}function hb(a){var b,c,d,e,f,h=a.nTHead,
+i=a.nTFoot,j=0===g("th, td",h).length,n=a.oClasses,m=a.aoColumns;j&&(e=g("<tr/>").appendTo(h));b=0;for(c=m.length;b<c;b++)f=m[b],d=g(f.nTh).addClass(f.sClass),j&&d.appendTo(e),a.oFeatures.bSort&&(d.addClass(f.sSortingClass),!1!==f.bSortable&&(d.attr("tabindex",a.iTabIndex).attr("aria-controls",a.sTableId),Ka(a,f.nTh,b))),f.sTitle!=d.html()&&d.html(f.sTitle),La(a,"header")(a,d,f,n);j&&$(a.aoHeader,h);g(h).find(">tr").attr("role","row");g(h).find(">tr>th, >tr>td").addClass(n.sHeaderTH);g(i).find(">tr>th, >tr>td").addClass(n.sFooterTH);
+if(null!==i){a=a.aoFooter[0];b=0;for(c=a.length;b<c;b++)f=m[b],f.nTf=a[b].cell,f.sClass&&g(f.nTf).addClass(f.sClass)}}function I(a,b,c){var d,e,f,h=[],i=[],j=a.aoColumns.length,n;if(b){c===l&&(c=!1);d=0;for(e=b.length;d<e;d++){h[d]=b[d].slice();h[d].nTr=b[d].nTr;for(f=j-1;0<=f;f--)!a.aoColumns[f].bVisible&&!c&&h[d].splice(f,1);i.push([])}d=0;for(e=h.length;d<e;d++){if(a=h[d].nTr)for(;f=a.firstChild;)a.removeChild(f);f=0;for(b=h[d].length;f<b;f++)if(n=j=1,i[d][f]===l){a.appendChild(h[d][f].cell);for(i[d][f]=
+1;h[d+j]!==l&&h[d][f].cell==h[d+j][f].cell;)i[d+j][f]=1,j++;for(;h[d][f+n]!==l&&h[d][f].cell==h[d][f+n].cell;){for(c=0;c<j;c++)i[d+c][f+n]=1;n++}g(h[d][f].cell).attr("rowspan",j).attr("colspan",n)}}}}function J(a){var b=t(a,"aoPreDrawCallback","preDraw",[a]);if(-1!==g.inArray(!1,b))C(a,!1);else{var b=[],c=0,d=a.asStripeClasses,e=d.length,f=a.oLanguage,h=a.iInitDisplayStart,i="ssp"==z(a),j=a.aiDisplay;a.bDrawing=!0;h!==l&&-1!==h&&(a._iDisplayStart=i?h:h>=a.fnRecordsDisplay()?0:h,a.iInitDisplayStart=
+-1);var h=a._iDisplayStart,n=a.fnDisplayEnd();if(a.bDeferLoading)a.bDeferLoading=!1,a.iDraw++,C(a,!1);else if(i){if(!a.bDestroying&&!ib(a))return}else a.iDraw++;if(0!==j.length){f=i?a.aoData.length:n;for(i=i?0:h;i<f;i++){var m=j[i],o=a.aoData[m];null===o.nTr&&Fa(a,m);m=o.nTr;if(0!==e){var k=d[c%e];o._sRowStripe!=k&&(g(m).removeClass(o._sRowStripe).addClass(k),o._sRowStripe=k)}t(a,"aoRowCallback",null,[m,o._aData,c,i]);b.push(m);c++}}else c=f.sZeroRecords,1==a.iDraw&&"ajax"==z(a)?c=f.sLoadingRecords:
+f.sEmptyTable&&0===a.fnRecordsTotal()&&(c=f.sEmptyTable),b[0]=g("<tr/>",{"class":e?d[0]:""}).append(g("<td />",{valign:"top",colSpan:Y(a),"class":a.oClasses.sRowEmpty}).html(c))[0];t(a,"aoHeaderCallback","header",[g(a.nTHead).children("tr")[0],Ha(a),h,n,j]);t(a,"aoFooterCallback","footer",[g(a.nTFoot).children("tr")[0],Ha(a),h,n,j]);d=g(a.nTBody);d.children().detach();d.append(g(b));t(a,"aoDrawCallback","draw",[a]);a.bSorted=!1;a.bFiltered=!1;a.bDrawing=!1}}function K(a,b){var c=a.oFeatures,d=c.bFilter;
+c.bSort&&jb(a);d?aa(a,a.oPreviousSearch):a.aiDisplay=a.aiDisplayMaster.slice();!0!==b&&(a._iDisplayStart=0);J(a)}function kb(a){var b=a.oClasses,c=g(a.nTable),c=g("<div/>").insertBefore(c),d=a.oFeatures,e=g("<div/>",{id:a.sTableId+"_wrapper","class":b.sWrapper+(a.nTFoot?"":" "+b.sNoFooter)});a.nHolding=c[0];a.nTableWrapper=e[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var f=a.sDom.split(""),h,i,j,n,m,o,k=0;k<f.length;k++){h=null;i=f[k];if("<"==i){j=g("<div/>")[0];n=f[k+1];if("'"==n||'"'==n){m=
+"";for(o=2;f[k+o]!=n;)m+=f[k+o],o++;"H"==m?m=b.sJUIHeader:"F"==m&&(m=b.sJUIFooter);-1!=m.indexOf(".")?(n=m.split("."),j.id=n[0].substr(1,n[0].length-1),j.className=n[1]):"#"==m.charAt(0)?j.id=m.substr(1,m.length-1):j.className=m;k+=o}e.append(j);e=g(j)}else if(">"==i)e=e.parent();else if("l"==i&&d.bPaginate&&d.bLengthChange)h=lb(a);else if("f"==i&&d.bFilter)h=mb(a);else if("r"==i&&d.bProcessing)h=nb(a);else if("t"==i)h=ob(a);else if("i"==i&&d.bInfo)h=pb(a);else if("p"==i&&d.bPaginate)h=qb(a);else if(0!==
+p.ext.feature.length){j=p.ext.feature;o=0;for(n=j.length;o<n;o++)if(i==j[o].cFeature){h=j[o].fnInit(a);break}}h&&(j=a.aanFeatures,j[i]||(j[i]=[]),j[i].push(h),e.append(h))}c.replaceWith(e)}function $(a,b){var c=g(b).children("tr"),d,e,f,h,i,j,n,m,o,k;a.splice(0,a.length);f=0;for(j=c.length;f<j;f++)a.push([]);f=0;for(j=c.length;f<j;f++){d=c[f];for(e=d.firstChild;e;){if("TD"==e.nodeName.toUpperCase()||"TH"==e.nodeName.toUpperCase()){m=1*e.getAttribute("colspan");o=1*e.getAttribute("rowspan");m=!m||
+0===m||1===m?1:m;o=!o||0===o||1===o?1:o;h=0;for(i=a[f];i[h];)h++;n=h;k=1===m?!0:!1;for(i=0;i<m;i++)for(h=0;h<o;h++)a[f+h][n+i]={cell:e,unique:k},a[f+h].nTr=d}e=e.nextSibling}}}function ma(a,b,c){var d=[];c||(c=a.aoHeader,b&&(c=[],$(c,b)));for(var b=0,e=c.length;b<e;b++)for(var f=0,h=c[b].length;f<h;f++)if(c[b][f].unique&&(!d[f]||!a.bSortCellsTop))d[f]=c[b][f].cell;return d}function na(a,b,c){t(a,"aoServerParams","serverParams",[b]);if(b&&g.isArray(b)){var d={},e=/(.*?)\[\]$/;g.each(b,function(a,b){var c=
+b.name.match(e);c?(c=c[0],d[c]||(d[c]=[]),d[c].push(b.value)):d[b.name]=b.value});b=d}var f,h=a.ajax,i=a.oInstance;if(g.isPlainObject(h)&&h.data){f=h.data;var j=g.isFunction(f)?f(b):f,b=g.isFunction(f)&&j?j:g.extend(!0,b,j);delete h.data}j={data:b,success:function(b){var d=b.error||b.sError;d&&a.oApi._fnLog(a,0,d);a.json=b;t(a,null,"xhr",[a,b]);c(b)},dataType:"json",cache:!1,type:a.sServerMethod,error:function(b,c){var d=a.oApi._fnLog;"parsererror"==c?d(a,0,"Invalid JSON response",1):4===b.readyState&&
+d(a,0,"Ajax error",7);C(a,!1)}};a.oAjaxData=b;t(a,null,"preXhr",[a,b]);a.fnServerData?a.fnServerData.call(i,a.sAjaxSource,g.map(b,function(a,b){return{name:b,value:a}}),c,a):a.sAjaxSource||"string"===typeof h?a.jqXHR=g.ajax(g.extend(j,{url:h||a.sAjaxSource})):g.isFunction(h)?a.jqXHR=h.call(i,b,c,a):(a.jqXHR=g.ajax(g.extend(j,h)),h.data=f)}function ib(a){if(a.bAjaxDataGet){a.iDraw++;C(a,!0);var b=rb(a);na(a,b,function(b){sb(a,b)},a);return!1}return!0}function rb(a){var b=a.aoColumns,c=b.length,d=a.oFeatures,
+e=a.oPreviousSearch,f=a.aoPreSearchCols,h,i=[],j,n,m,o=Q(a);h=a._iDisplayStart;j=!1!==d.bPaginate?a._iDisplayLength:-1;var k=function(a,b){i.push({name:a,value:b})};k("sEcho",a.iDraw);k("iColumns",c);k("sColumns",B(b,"sName").join(","));k("iDisplayStart",h);k("iDisplayLength",j);var l={draw:a.iDraw,columns:[],order:[],start:h,length:j,search:{value:e.sSearch,regex:e.bRegex}};for(h=0;h<c;h++)n=b[h],m=f[h],j="function"==typeof n.mData?"function":n.mData,l.columns.push({data:j,name:n.sName,searchable:n.bSearchable,
+orderable:n.bSortable,search:{value:m.sSearch,regex:m.bRegex}}),k("mDataProp_"+h,j),d.bFilter&&(k("sSearch_"+h,m.sSearch),k("bRegex_"+h,m.bRegex),k("bSearchable_"+h,n.bSearchable)),d.bSort&&k("bSortable_"+h,n.bSortable);d.bFilter&&(k("sSearch",e.sSearch),k("bRegex",e.bRegex));d.bSort&&(g.each(o,function(a,b){l.order.push({column:b.col,dir:b.dir});k("iSortCol_"+a,b.col);k("sSortDir_"+a,b.dir)}),k("iSortingCols",o.length));b=p.ext.legacy.ajax;return null===b?a.sAjaxSource?i:l:b?i:l}function sb(a,b){var c=
+b.sEcho!==l?b.sEcho:b.draw,d=b.iTotalRecords!==l?b.iTotalRecords:b.recordsTotal,e=b.iTotalDisplayRecords!==l?b.iTotalDisplayRecords:b.recordsFiltered;if(c){if(1*c<a.iDraw)return;a.iDraw=1*c}ja(a);a._iRecordsTotal=parseInt(d,10);a._iRecordsDisplay=parseInt(e,10);c=oa(a,b);d=0;for(e=c.length;d<e;d++)H(a,c[d]);a.aiDisplay=a.aiDisplayMaster.slice();a.bAjaxDataGet=!1;J(a);a._bInitComplete||pa(a,b);a.bAjaxDataGet=!0;C(a,!1)}function oa(a,b){var c=g.isPlainObject(a.ajax)&&a.ajax.dataSrc!==l?a.ajax.dataSrc:
+a.sAjaxDataProp;return"data"===c?b.aaData||b[c]:""!==c?T(c)(b):b}function mb(a){var b=a.oClasses,c=a.sTableId,d=a.oPreviousSearch,e=a.aanFeatures,f='<input type="search" class="'+b.sFilterInput+'"/>',h=a.oLanguage.sSearch,h=h.match(/_INPUT_/)?h.replace("_INPUT_",f):h+f,b=g("<div/>",{id:!e.f?c+"_filter":null,"class":b.sFilter}).append(g("<label/>").append(h)),e=function(){var b=!this.value?"":this.value;b!=d.sSearch&&(aa(a,{sSearch:b,bRegex:d.bRegex,bSmart:d.bSmart,bCaseInsensitive:d.bCaseInsensitive}),
+a._iDisplayStart=0,J(a))},i=g("input",b).val(d.sSearch.replace('"',"&quot;")).bind("keyup.DT search.DT input.DT paste.DT cut.DT","ssp"===z(a)?Ma(e,400):e).bind("keypress.DT",function(a){if(13==a.keyCode)return!1}).attr("aria-controls",c);g(a.nTable).on("filter.DT",function(){try{i[0]!==N.activeElement&&i.val(d.sSearch)}catch(a){}});return b[0]}function aa(a,b,c){var d=a.oPreviousSearch,e=a.aoPreSearchCols,f=function(a){d.sSearch=a.sSearch;d.bRegex=a.bRegex;d.bSmart=a.bSmart;d.bCaseInsensitive=a.bCaseInsensitive};
+Da(a);if("ssp"!=z(a)){tb(a,b.sSearch,c,b.bEscapeRegex!==l?!b.bEscapeRegex:b.bRegex,b.bSmart,b.bCaseInsensitive);f(b);for(b=0;b<e.length;b++)ub(a,e[b].sSearch,b,e[b].bEscapeRegex!==l?!e[b].bEscapeRegex:e[b].bRegex,e[b].bSmart,e[b].bCaseInsensitive);vb(a)}else f(b);a.bFiltered=!0;t(a,null,"search",[a])}function vb(a){for(var b=p.ext.search,c=a.aiDisplay,d,e,f=0,h=b.length;f<h;f++)for(var i=c.length-1;0<=i;i--)e=c[i],d=a.aoData[e],b[f](a,d._aFilterData,e,d._aData)||c.splice(i,1)}function ub(a,b,c,d,
+e,f){if(""!==b)for(var h=a.aiDisplay,d=Na(b,d,e,f),e=h.length-1;0<=e;e--)b=a.aoData[h[e]]._aFilterData[c],d.test(b)||h.splice(e,1)}function tb(a,b,c,d,e,f){var d=Na(b,d,e,f),e=a.oPreviousSearch.sSearch,f=a.aiDisplayMaster,h;0!==p.ext.search.length&&(c=!0);h=wb(a);if(0>=b.length)a.aiDisplay=f.slice();else{if(h||c||e.length>b.length||0!==b.indexOf(e)||a.bSorted)a.aiDisplay=f.slice();b=a.aiDisplay;for(c=b.length-1;0<=c;c--)d.test(a.aoData[b[c]]._sFilterRow)||b.splice(c,1)}}function Na(a,b,c,d){a=b?a:
+Oa(a);c&&(a="^(?=.*?"+g.map(a.match(/"[^"]+"|[^ ]+/g)||"",function(a){return'"'===a.charAt(0)?a.match(/^"(.*)"$/)[1]:a}).join(")(?=.*?")+").*$");return RegExp(a,d?"i":"")}function Oa(a){return a.replace(Sb,"\\$1")}function wb(a){var b=a.aoColumns,c,d,e,f,h,i,j,g,m=p.ext.type.search;c=!1;d=0;for(f=a.aoData.length;d<f;d++)if(g=a.aoData[d],!g._aFilterData){i=[];e=0;for(h=b.length;e<h;e++)c=b[e],c.bSearchable?(j=A(a,d,e,"filter"),j=m[c.sType]?m[c.sType](j):null!==j?j:""):j="",j.indexOf&&-1!==j.indexOf("&")&&
+(qa.innerHTML=j,j=Tb?qa.textContent:qa.innerText),j.replace&&(j=j.replace(/[\r\n]/g,"")),i.push(j);g._aFilterData=i;g._sFilterRow=i.join("  ");c=!0}return c}function pb(a){var b=a.sTableId,c=a.aanFeatures.i,d=g("<div/>",{"class":a.oClasses.sInfo,id:!c?b+"_info":null});c||(a.aoDrawCallback.push({fn:xb,sName:"information"}),d.attr("role","status").attr("aria-live","polite"),g(a.nTable).attr("aria-describedby",b+"_info"));return d[0]}function xb(a){var b=a.aanFeatures.i;if(0!==b.length){var c=a.oLanguage,
+d=a._iDisplayStart+1,e=a.fnDisplayEnd(),f=a.fnRecordsTotal(),h=a.fnRecordsDisplay(),i=h?c.sInfo:c.sInfoEmpty;h!==f&&(i+=" "+c.sInfoFiltered);i+=c.sInfoPostFix;i=yb(a,i);c=c.fnInfoCallback;null!==c&&(i=c.call(a.oInstance,a,d,e,f,h,i));g(b).html(i)}}function yb(a,b){var c=a.fnFormatNumber,d=a._iDisplayStart+1,e=a._iDisplayLength,f=a.fnRecordsDisplay(),h=-1===e;return b.replace(/_START_/g,c.call(a,d)).replace(/_END_/g,c.call(a,a.fnDisplayEnd())).replace(/_MAX_/g,c.call(a,a.fnRecordsTotal())).replace(/_TOTAL_/g,
+c.call(a,f)).replace(/_PAGE_/g,c.call(a,h?1:Math.ceil(d/e))).replace(/_PAGES_/g,c.call(a,h?1:Math.ceil(f/e)))}function ra(a){var b,c,d=a.iInitDisplayStart,e=a.aoColumns,f;c=a.oFeatures;if(a.bInitialised){kb(a);hb(a);I(a,a.aoHeader);I(a,a.aoFooter);C(a,!0);c.bAutoWidth&&Ca(a);b=0;for(c=e.length;b<c;b++)f=e[b],f.sWidth&&(f.nTh.style.width=s(f.sWidth));K(a);e=z(a);"ssp"!=e&&("ajax"==e?na(a,[],function(c){var f=oa(a,c);for(b=0;b<f.length;b++)H(a,f[b]);a.iInitDisplayStart=d;K(a);C(a,!1);pa(a,c)},a):(C(a,
+!1),pa(a)))}else setTimeout(function(){ra(a)},200)}function pa(a,b){a._bInitComplete=!0;b&&U(a);t(a,"aoInitComplete","init",[a,b])}function Pa(a,b){var c=parseInt(b,10);a._iDisplayLength=c;Qa(a);t(a,null,"length",[a,c])}function lb(a){for(var b=a.oClasses,c=a.sTableId,d=a.aLengthMenu,e=g.isArray(d[0]),f=e?d[0]:d,e=e?d[1]:d,d=g("<select/>",{name:c+"_length","aria-controls":c,"class":b.sLengthSelect}),h=0,i=f.length;h<i;h++)d[0][h]=new Option(e[h],f[h]);var j=g("<div><label/></div>").addClass(b.sLength);
+a.aanFeatures.l||(j[0].id=c+"_length");b=a.oLanguage.sLengthMenu.split(/(_MENU_)/);j.children().append(1<b.length?[b[0],d,b[2]]:b[0]);g("select",j).val(a._iDisplayLength).bind("change.DT",function(){Pa(a,g(this).val());J(a)});g(a.nTable).bind("length.dt.DT",function(a,b,c){g("select",j).val(c)});return j[0]}function qb(a){var b=a.sPaginationType,c=p.ext.pager[b],d="function"===typeof c,e=function(a){J(a)},b=g("<div/>").addClass(a.oClasses.sPaging+b)[0],f=a.aanFeatures;d||c.fnInit(a,b,e);f.p||(b.id=
+a.sTableId+"_paginate",a.aoDrawCallback.push({fn:function(a){if(d){var b=a._iDisplayStart,j=a._iDisplayLength,g=a.fnRecordsDisplay(),m=-1===j,b=m?0:Math.ceil(b/j),j=m?1:Math.ceil(g/j),g=c(b,j),o,m=0;for(o=f.p.length;m<o;m++)La(a,"pageButton")(a,f.p[m],m,g,b,j)}else c.fnUpdate(a,e)},sName:"pagination"}));return b}function Ra(a,b,c){var d=a._iDisplayStart,e=a._iDisplayLength,f=a.fnRecordsDisplay();0===f||-1===e?d=0:"number"===typeof b?(d=b*e,d>f&&(d=0)):"first"==b?d=0:"previous"==b?(d=0<=e?d-e:0,0>
+d&&(d=0)):"next"==b?d+e<f&&(d+=e):"last"==b?d=Math.floor((f-1)/e)*e:O(a,0,"Unknown paging action: "+b,5);b=a._iDisplayStart!==d;a._iDisplayStart=d;b&&(t(a,null,"page",[a]),c&&J(a));return b}function nb(a){return g("<div/>",{id:!a.aanFeatures.r?a.sTableId+"_processing":null,"class":a.oClasses.sProcessing}).html(a.oLanguage.sProcessing).insertBefore(a.nTable)[0]}function C(a,b){a.oFeatures.bProcessing&&g(a.aanFeatures.r).css("display",b?"block":"none");t(a,null,"processing",[a,b])}function ob(a){var b=
+g(a.nTable);b.attr("role","grid");var c=a.oScroll;if(""===c.sX&&""===c.sY)return a.nTable;var d=c.sX,e=c.sY,f=a.oClasses,h=b.children("caption"),i=h.length?h[0]._captionSide:null,j=g(b[0].cloneNode(!1)),n=g(b[0].cloneNode(!1)),m=b.children("tfoot");c.sX&&"100%"===b.attr("width")&&b.removeAttr("width");m.length||(m=null);c=g("<div/>",{"class":f.sScrollWrapper}).append(g("<div/>",{"class":f.sScrollHead}).css({overflow:"hidden",position:"relative",border:0,width:d?!d?null:s(d):"100%"}).append(g("<div/>",
+{"class":f.sScrollHeadInner}).css({"box-sizing":"content-box",width:c.sXInner||"100%"}).append(j.removeAttr("id").css("margin-left",0).append(b.children("thead")))).append("top"===i?h:null)).append(g("<div/>",{"class":f.sScrollBody}).css({overflow:"auto",height:!e?null:s(e),width:!d?null:s(d)}).append(b));m&&c.append(g("<div/>",{"class":f.sScrollFoot}).css({overflow:"hidden",border:0,width:d?!d?null:s(d):"100%"}).append(g("<div/>",{"class":f.sScrollFootInner}).append(n.removeAttr("id").css("margin-left",
+0).append(b.children("tfoot")))).append("bottom"===i?h:null));var b=c.children(),o=b[0],f=b[1],k=m?b[2]:null;d&&g(f).scroll(function(){var a=this.scrollLeft;o.scrollLeft=a;m&&(k.scrollLeft=a)});a.nScrollHead=o;a.nScrollBody=f;a.nScrollFoot=k;a.aoDrawCallback.push({fn:V,sName:"scrolling"});return c[0]}function V(a){var b=a.oScroll,c=b.sX,d=b.sXInner,e=b.sY,f=b.iBarWidth,h=g(a.nScrollHead),i=h[0].style,j=h.children("div"),n=j[0].style,m=j.children("table"),j=a.nScrollBody,o=g(j),k=j.style,l=g(a.nScrollFoot).children("div"),
+p=l.children("table"),r=g(a.nTHead),q=g(a.nTable),ba=q[0],L=ba.style,t=a.nTFoot?g(a.nTFoot):null,ca=a.oBrowser,v=ca.bScrollOversize,x,u,y,w,z,A=[],B=[],C=[],D,E=function(a){a=a.style;a.paddingTop="0";a.paddingBottom="0";a.borderTopWidth="0";a.borderBottomWidth="0";a.height=0};q.children("thead, tfoot").remove();z=r.clone().prependTo(q);x=r.find("tr");y=z.find("tr");z.find("th, td").removeAttr("tabindex");t&&(w=t.clone().prependTo(q),u=t.find("tr"),w=w.find("tr"));c||(k.width="100%",h[0].style.width=
+"100%");g.each(ma(a,z),function(b,c){D=ga(a,b);c.style.width=a.aoColumns[D].sWidth});t&&F(function(a){a.style.width=""},w);b.bCollapse&&""!==e&&(k.height=o[0].offsetHeight+r[0].offsetHeight+"px");h=q.outerWidth();if(""===c){if(L.width="100%",v&&(q.find("tbody").height()>j.offsetHeight||"scroll"==o.css("overflow-y")))L.width=s(q.outerWidth()-f)}else""!==d?L.width=s(d):h==o.width()&&o.height()<q.height()?(L.width=s(h-f),q.outerWidth()>h-f&&(L.width=s(h))):L.width=s(h);h=q.outerWidth();F(E,y);F(function(a){C.push(a.innerHTML);
+A.push(s(g(a).css("width")))},y);F(function(a,b){a.style.width=A[b]},x);g(y).height(0);t&&(F(E,w),F(function(a){B.push(s(g(a).css("width")))},w),F(function(a,b){a.style.width=B[b]},u),g(w).height(0));F(function(a,b){a.innerHTML='<div class="dataTables_sizing" style="height:0;overflow:hidden;">'+C[b]+"</div>";a.style.width=A[b]},y);t&&F(function(a,b){a.innerHTML="";a.style.width=B[b]},w);if(q.outerWidth()<h){u=j.scrollHeight>j.offsetHeight||"scroll"==o.css("overflow-y")?h+f:h;if(v&&(j.scrollHeight>
+j.offsetHeight||"scroll"==o.css("overflow-y")))L.width=s(u-f);(""===c||""!==d)&&O(a,1,"Possible column misalignment",6)}else u="100%";k.width=s(u);i.width=s(u);t&&(a.nScrollFoot.style.width=s(u));!e&&v&&(k.height=s(ba.offsetHeight+f));e&&b.bCollapse&&(k.height=s(e),b=c&&ba.offsetWidth>j.offsetWidth?f:0,ba.offsetHeight<j.offsetHeight&&(k.height=s(ba.offsetHeight+b)));b=q.outerWidth();m[0].style.width=s(b);n.width=s(b);m=q.height()>j.clientHeight||"scroll"==o.css("overflow-y");ca="padding"+(ca.bScrollbarLeft?
+"Left":"Right");n[ca]=m?f+"px":"0px";t&&(p[0].style.width=s(b),l[0].style.width=s(b),l[0].style[ca]=m?f+"px":"0px");o.scroll();if(a.bSorted||a.bFiltered)j.scrollTop=0}function F(a,b,c){for(var d=0,e=0,f=b.length,h,i;e<f;){h=b[e].firstChild;for(i=c?c[e].firstChild:null;h;)1===h.nodeType&&(c?a(h,i,d):a(h,d),d++),h=h.nextSibling,i=c?i.nextSibling:null;e++}}function Ca(a){var b=a.nTable,c=a.aoColumns,d=a.oScroll,e=d.sY,f=d.sX,h=d.sXInner,i=c.length,d=W(a,"bVisible"),j=g("th",a.nTHead),n=b.getAttribute("width"),
+m=b.parentNode,o=!1,k,l;for(k=0;k<d.length;k++)l=c[d[k]],null!==l.sWidth&&(l.sWidth=zb(l.sWidthOrig,m),o=!0);if(!o&&!f&&!e&&i==Y(a)&&i==j.length)for(k=0;k<i;k++)c[k].sWidth=s(j.eq(k).width());else{i=g(b.cloneNode(!1)).css("visibility","hidden").removeAttr("id").append(g(a.nTHead).clone(!1)).append(g(a.nTFoot).clone(!1)).append(g("<tbody><tr/></tbody>"));i.find("tfoot th, tfoot td").css("width","");var p=i.find("tbody tr"),j=ma(a,i.find("thead")[0]);for(k=0;k<d.length;k++)l=c[d[k]],j[k].style.width=
+null!==l.sWidthOrig&&""!==l.sWidthOrig?s(l.sWidthOrig):"";if(a.aoData.length)for(k=0;k<d.length;k++)o=d[k],l=c[o],g(Ab(a,o)).clone(!1).append(l.sContentPadding).appendTo(p);i.appendTo(m);f&&h?i.width(h):f?(i.css("width","auto"),i.width()<m.offsetWidth&&i.width(m.offsetWidth)):e?i.width(m.offsetWidth):n&&i.width(n);Bb(a,i[0]);if(f){for(k=h=0;k<d.length;k++)l=c[d[k]],e=g(j[k]).outerWidth(),h+=null===l.sWidthOrig?e:parseInt(l.sWidth,10)+e-g(j[k]).width();i.width(s(h));b.style.width=s(h)}for(k=0;k<d.length;k++)if(l=
+c[d[k]],e=g(j[k]).width())l.sWidth=s(e);b.style.width=s(i.css("width"));i.remove()}n&&(b.style.width=s(n));if((n||f)&&!a._reszEvt)g(za).bind("resize.DT-"+a.sInstance,Ma(function(){U(a)})),a._reszEvt=!0}function Ma(a,b){var c=b||200,d,e;return function(){var b=this,h=+new Date,i=arguments;d&&h<d+c?(clearTimeout(e),e=setTimeout(function(){d=l;a.apply(b,i)},c)):d?(d=h,a.apply(b,i)):d=h}}function zb(a,b){if(!a)return 0;var c=g("<div/>").css("width",s(a)).appendTo(b||N.body),d=c[0].offsetWidth;c.remove();
+return d}function Bb(a,b){var c=a.oScroll;if(c.sX||c.sY)c=!c.sX?c.iBarWidth:0,b.style.width=s(g(b).outerWidth()-c)}function Ab(a,b){var c=Cb(a,b);if(0>c)return null;var d=a.aoData[c];return!d.nTr?g("<td/>").html(A(a,c,b,"display"))[0]:d.anCells[b]}function Cb(a,b){for(var c,d=-1,e=-1,f=0,h=a.aoData.length;f<h;f++)c=A(a,f,b,"display")+"",c=c.replace(Ub,""),c.length>d&&(d=c.length,e=f);return e}function s(a){return null===a?"0px":"number"==typeof a?0>a?"0px":a+"px":a.match(/\d$/)?a+"px":a}function Db(){if(!p.__scrollbarWidth){var a=
+g("<p/>").css({width:"100%",height:200,padding:0})[0],b=g("<div/>").css({position:"absolute",top:0,left:0,width:200,height:150,padding:0,overflow:"hidden",visibility:"hidden"}).append(a).appendTo("body"),c=a.offsetWidth;b.css("overflow","scroll");a=a.offsetWidth;c===a&&(a=b[0].clientWidth);b.remove();p.__scrollbarWidth=c-a}return p.__scrollbarWidth}function Q(a){var b,c,d=[],e=a.aoColumns,f,h,i,j;b=a.aaSortingFixed;c=g.isPlainObject(b);var n=[];f=function(a){a.length&&!g.isArray(a[0])?n.push(a):n.push.apply(n,
+a)};g.isArray(b)&&f(b);c&&b.pre&&f(b.pre);f(a.aaSorting);c&&b.post&&f(b.post);for(a=0;a<n.length;a++){j=n[a][0];f=e[j].aDataSort;b=0;for(c=f.length;b<c;b++)h=f[b],i=e[h].sType||"string",d.push({src:j,col:h,dir:n[a][1],index:n[a][2],type:i,formatter:p.ext.type.order[i+"-pre"]})}return d}function jb(a){var b,c,d=[],e=p.ext.type.order,f=a.aoData,h=0,i,g=a.aiDisplayMaster,n;Da(a);n=Q(a);b=0;for(c=n.length;b<c;b++)i=n[b],i.formatter&&h++,Eb(a,i.col);if("ssp"!=z(a)&&0!==n.length){b=0;for(c=g.length;b<c;b++)d[g[b]]=
+b;h===n.length?g.sort(function(a,b){var c,e,h,g,i=n.length,j=f[a]._aSortData,l=f[b]._aSortData;for(h=0;h<i;h++)if(g=n[h],c=j[g.col],e=l[g.col],c=c<e?-1:c>e?1:0,0!==c)return"asc"===g.dir?c:-c;c=d[a];e=d[b];return c<e?-1:c>e?1:0}):g.sort(function(a,b){var c,h,g,i,j=n.length,l=f[a]._aSortData,p=f[b]._aSortData;for(g=0;g<j;g++)if(i=n[g],c=l[i.col],h=p[i.col],i=e[i.type+"-"+i.dir]||e["string-"+i.dir],c=i(c,h),0!==c)return c;c=d[a];h=d[b];return c<h?-1:c>h?1:0})}a.bSorted=!0}function Fb(a){for(var b,c,
+d=a.aoColumns,e=Q(a),a=a.oLanguage.oAria,f=0,h=d.length;f<h;f++){c=d[f];var i=c.asSorting;b=c.sTitle.replace(/<.*?>/g,"");var g=c.nTh;g.removeAttribute("aria-sort");c.bSortable&&(0<e.length&&e[0].col==f?(g.setAttribute("aria-sort","asc"==e[0].dir?"ascending":"descending"),c=i[e[0].index+1]||i[0]):c=i[0],b+="asc"===c?a.sSortAscending:a.sSortDescending);g.setAttribute("aria-label",b)}}function Sa(a,b,c,d){var e=a.aaSorting,f=a.aoColumns[b].asSorting,h=function(a){var b=a._idx;b===l&&(b=g.inArray(a[1],
+f));return b+1>=f.length?0:b+1};c&&a.oFeatures.bSortMulti?(c=g.inArray(b,B(e,"0")),-1!==c?(b=h(e[c]),e[c][1]=f[b],e[c]._idx=b):(e.push([b,f[0],0]),e[e.length-1]._idx=0)):e.length&&e[0][0]==b?(b=h(e[0]),e.length=1,e[0][1]=f[b],e[0]._idx=b):(e.length=0,e.push([b,f[0]]),e[0]._idx=0);K(a);"function"==typeof d&&d(a)}function Ka(a,b,c,d){var e=a.aoColumns[c];Ta(b,{},function(b){!1!==e.bSortable&&(a.oFeatures.bProcessing?(C(a,!0),setTimeout(function(){Sa(a,c,b.shiftKey,d);"ssp"!==z(a)&&C(a,!1)},0)):Sa(a,
+c,b.shiftKey,d))})}function sa(a){var b=a.aLastSort,c=a.oClasses.sSortColumn,d=Q(a),e=a.oFeatures,f,h;if(e.bSort&&e.bSortClasses){e=0;for(f=b.length;e<f;e++)h=b[e].src,g(B(a.aoData,"anCells",h)).removeClass(c+(2>e?e+1:3));e=0;for(f=d.length;e<f;e++)h=d[e].src,g(B(a.aoData,"anCells",h)).addClass(c+(2>e?e+1:3))}a.aLastSort=d}function Eb(a,b){var c=a.aoColumns[b],d=p.ext.order[c.sSortDataType],e;d&&(e=d.call(a.oInstance,a,b,X(a,b)));for(var f,h=p.ext.type.order[c.sType+"-pre"],g=0,j=a.aoData.length;g<
+j;g++)if(c=a.aoData[g],c._aSortData||(c._aSortData=[]),!c._aSortData[b]||d)f=d?e[g]:A(a,g,b,"sort"),c._aSortData[b]=h?h(f):f}function ta(a){if(a.oFeatures.bStateSave&&!a.bDestroying){var b={iCreate:+new Date,iStart:a._iDisplayStart,iLength:a._iDisplayLength,aaSorting:g.extend(!0,[],a.aaSorting),oSearch:g.extend(!0,{},a.oPreviousSearch),aoSearchCols:g.extend(!0,[],a.aoPreSearchCols),abVisCols:B(a.aoColumns,"bVisible")};t(a,"aoStateSaveParams","stateSaveParams",[a,b]);a.fnStateSaveCallback.call(a.oInstance,
+a,b)}}function Gb(a){var b,c,d=a.aoColumns;if(a.oFeatures.bStateSave){var e=a.fnStateLoadCallback.call(a.oInstance,a);if(e&&(b=t(a,"aoStateLoadParams","stateLoadParams",[a,e]),-1===g.inArray(!1,b)&&(b=a.iStateDuration,!(0<b&&e.iCreate<+new Date-1E3*b)&&d.length===e.aoSearchCols.length))){a.oLoadedState=g.extend(!0,{},e);a._iDisplayStart=e.iStart;a.iInitDisplayStart=e.iStart;a._iDisplayLength=e.iLength;a.aaSorting=g.map(e.aaSorting,function(a){return a[0]>=d.length?[0,a[1]]:a});g.extend(a.oPreviousSearch,
+e.oSearch);g.extend(!0,a.aoPreSearchCols,e.aoSearchCols);var f=e.abVisCols;b=0;for(c=f.length;b<c;b++)d[b].bVisible=f[b];t(a,"aoStateLoaded","stateLoaded",[a,e])}}}function ua(a){var b=p.settings,a=g.inArray(a,B(b,"nTable"));return-1!==a?b[a]:null}function O(a,b,c,d){c="DataTables warning: "+(null!==a?"table id="+a.sTableId+" - ":"")+c;d&&(c+=". For more information about this error, please see http://datatables.net/tn/"+d);if(b)za.console&&console.log&&console.log(c);else if(a=p.ext,"alert"==(a.sErrMode||
+a.errMode))alert(c);else throw Error(c);}function D(a,b,c,d){g.isArray(c)?g.each(c,function(c,d){g.isArray(d)?D(a,b,d[0],d[1]):D(a,b,d)}):(d===l&&(d=c),b[c]!==l&&(a[d]=b[c]))}function Hb(a,b,c){var d,e;for(e in b)b.hasOwnProperty(e)&&(d=b[e],g.isPlainObject(d)?(g.isPlainObject(a[e])||(a[e]={}),g.extend(!0,a[e],d)):a[e]=c&&"data"!==e&&"aaData"!==e&&g.isArray(d)?d.slice():d);return a}function Ta(a,b,c){g(a).bind("click.DT",b,function(b){a.blur();c(b)}).bind("keypress.DT",b,function(a){13===a.which&&
+(a.preventDefault(),c(a))}).bind("selectstart.DT",function(){return!1})}function y(a,b,c,d){c&&a[b].push({fn:c,sName:d})}function t(a,b,c,d){var e=[];b&&(e=g.map(a[b].slice().reverse(),function(b){return b.fn.apply(a.oInstance,d)}));null!==c&&g(a.nTable).trigger(c+".dt",d);return e}function Qa(a){var b=a._iDisplayStart,c=a.fnDisplayEnd(),d=a._iDisplayLength;c===a.fnRecordsDisplay()&&(b=c-d);if(-1===d||0>b)b=0;a._iDisplayStart=b}function La(a,b){var c=a.renderer,d=p.ext.renderer[b];return g.isPlainObject(c)&&
+c[b]?d[c[b]]||d._:"string"===typeof c?d[c]||d._:d._}function z(a){return a.oFeatures.bServerSide?"ssp":a.ajax||a.sAjaxSource?"ajax":"dom"}function Ua(a,b){var c=[],c=Ib.numbers_length,d=Math.floor(c/2);b<=c?c=R(0,b):a<=d?(c=R(0,c-2),c.push("ellipsis"),c.push(b-1)):(a>=b-1-d?c=R(b-(c-2),b):(c=R(a-1,a+2),c.push("ellipsis"),c.push(b-1)),c.splice(0,0,"ellipsis"),c.splice(0,0,0));c.DT_el="span";return c}function bb(a){g.each({num:function(b){return va(b,a)},"num-fmt":function(b){return va(b,a,Va)},"html-num":function(b){return va(b,
+a,wa)},"html-num-fmt":function(b){return va(b,a,wa,Va)}},function(b,c){u.type.order[b+a+"-pre"]=c})}function Jb(a){return function(){var b=[ua(this[p.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return p.ext.internal[a].apply(this,b)}}var p,u,q,r,x,Wa={},Kb=/[\r\n]/g,wa=/<.*?>/g,Vb=/^[\d\+\-a-zA-Z]/,Sb=RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)","g"),Va=/[',$\u00a3\u20ac\u00a5%\u2009\u202F]/g,da=function(a){return!a||"-"===a?!0:!1},Lb=function(a){var b=
+parseInt(a,10);return!isNaN(b)&&isFinite(a)?b:null},Mb=function(a,b){Wa[b]||(Wa[b]=RegExp(Oa(b),"g"));return"string"===typeof a?a.replace(/\./g,"").replace(Wa[b],"."):a},Xa=function(a,b,c){var d="string"===typeof a;b&&d&&(a=Mb(a,b));c&&d&&(a=a.replace(Va,""));return!a||"-"===a||!isNaN(parseFloat(a))&&isFinite(a)},Nb=function(a,b,c){return da(a)?!0:a&&"string"!==typeof a?null:Xa(a.replace(wa,""),b,c)?!0:null},B=function(a,b,c){var d=[],e=0,f=a.length;if(c!==l)for(;e<f;e++)a[e]&&a[e][b]&&d.push(a[e][b][c]);
+else for(;e<f;e++)a[e]&&d.push(a[e][b]);return d},xa=function(a,b,c,d){var e=[],f=0,h=b.length;if(d!==l)for(;f<h;f++)e.push(a[b[f]][c][d]);else for(;f<h;f++)e.push(a[b[f]][c]);return e},R=function(a,b){var c=[],d;b===l?(b=0,d=a):(d=b,b=a);for(var e=b;e<d;e++)c.push(e);return c},Ja=function(a){var b=[],c,d,e=a.length,f,h=0;d=0;a:for(;d<e;d++){c=a[d];for(f=0;f<h;f++)if(b[f]===c)continue a;b.push(c);h++}return b},w=function(a,b,c){a[b]!==l&&(a[c]=a[b])},Z=/\[.*?\]$/,P=/\(\)$/,qa=g("<div>")[0],Tb=qa.textContent!==
+l,Ub=/<.*?>/g;p=function(a){this.$=function(a,b){return this.api(!0).$(a,b)};this._=function(a,b){return this.api(!0).rows(a,b).data()};this.api=function(a){return a?new q(ua(this[u.iApiIndex])):new q(this)};this.fnAddData=function(a,b){var c=this.api(!0),d=g.isArray(a)&&(g.isArray(a[0])||g.isPlainObject(a[0]))?c.rows.add(a):c.row.add(a);(b===l||b)&&c.draw();return d.flatten().toArray()};this.fnAdjustColumnSizing=function(a){var b=this.api(!0).columns.adjust(),c=b.settings()[0],d=c.oScroll;a===l||
+a?b.draw(!1):(""!==d.sX||""!==d.sY)&&V(c)};this.fnClearTable=function(a){var b=this.api(!0).clear();(a===l||a)&&b.draw()};this.fnClose=function(a){this.api(!0).row(a).child.hide()};this.fnDeleteRow=function(a,b,c){var d=this.api(!0),a=d.rows(a),e=a.settings()[0],g=e.aoData[a[0][0]];a.remove();b&&b.call(this,e,g);(c===l||c)&&d.draw();return g};this.fnDestroy=function(a){this.api(!0).destroy(a)};this.fnDraw=function(a){this.api(!0).draw(!a)};this.fnFilter=function(a,b,c,d,e,g){e=this.api(!0);null===
+b||b===l?e.search(a,c,d,g):e.column(b).search(a,c,d,g);e.draw()};this.fnGetData=function(a,b){var c=this.api(!0);if(a!==l){var d=a.nodeName?a.nodeName.toLowerCase():"";return b!==l||"td"==d||"th"==d?c.cell(a,b).data():c.row(a).data()||null}return c.data().toArray()};this.fnGetNodes=function(a){var b=this.api(!0);return a!==l?b.row(a).node():b.rows().nodes().flatten().toArray()};this.fnGetPosition=function(a){var b=this.api(!0),c=a.nodeName.toUpperCase();return"TR"==c?b.row(a).index():"TD"==c||"TH"==
+c?(a=b.cell(a).index(),[a.row,a.columnVisible,a.column]):null};this.fnIsOpen=function(a){return this.api(!0).row(a).child.isShown()};this.fnOpen=function(a,b,c){return this.api(!0).row(a).child(b,c).show().child()[0]};this.fnPageChange=function(a,b){var c=this.api(!0).page(a);(b===l||b)&&c.draw(!1)};this.fnSetColumnVis=function(a,b,c){a=this.api(!0).column(a).visible(b);(c===l||c)&&a.columns.adjust().draw()};this.fnSettings=function(){return ua(this[u.iApiIndex])};this.fnSort=function(a){this.api(!0).order(a).draw()};
+this.fnSortListener=function(a,b,c){this.api(!0).order.listener(a,b,c)};this.fnUpdate=function(a,b,c,d,e){var g=this.api(!0);c===l||null===c?g.row(b).data(a):g.cell(b,c).data(a);(e===l||e)&&g.columns.adjust();(d===l||d)&&g.draw();return 0};this.fnVersionCheck=u.fnVersionCheck;var b=this,c=a===l,d=this.length;c&&(a={});this.oApi=this.internal=u.internal;for(var e in p.ext.internal)e&&(this[e]=Jb(e));this.each(function(){var e={},h=1<d?Hb(e,a,!0):a,i=0,j,n=this.getAttribute("id"),e=!1,m=p.defaults;
+if("table"!=this.nodeName.toLowerCase())O(null,0,"Non-table node initialisation ("+this.nodeName+")",2);else{cb(m);db(m.column);G(m,m,!0);G(m.column,m.column,!0);G(m,h);var o=p.settings,i=0;for(j=o.length;i<j;i++){if(o[i].nTable==this){j=h.bRetrieve!==l?h.bRetrieve:m.bRetrieve;if(c||j)return o[i].oInstance;if(h.bDestroy!==l?h.bDestroy:m.bDestroy){o[i].oInstance.fnDestroy();break}else{O(o[i],0,"Cannot reinitialise DataTable",3);return}}if(o[i].sTableId==this.id){o.splice(i,1);break}}if(null===n||""===
+n)this.id=n="DataTables_Table_"+p.ext._unique++;var k=g.extend(!0,{},p.models.oSettings,{nTable:this,oApi:b.internal,oInit:h,sDestroyWidth:g(this)[0].style.width,sInstance:n,sTableId:n});o.push(k);k.oInstance=1===b.length?b:g(this).dataTable();cb(h);h.oLanguage&&M(h.oLanguage);h.aLengthMenu&&!h.iDisplayLength&&(h.iDisplayLength=g.isArray(h.aLengthMenu[0])?h.aLengthMenu[0][0]:h.aLengthMenu[0]);h=Hb(g.extend(!0,{},m),h);D(k.oFeatures,h,"bPaginate bLengthChange bFilter bSort bSortMulti bInfo bProcessing bAutoWidth bSortClasses bServerSide bDeferRender".split(" "));
+D(k,h,["asStripeClasses","ajax","fnServerData","fnFormatNumber","sServerMethod","aaSorting","aaSortingFixed","aLengthMenu","sPaginationType","sAjaxSource","sAjaxDataProp","iStateDuration","sDom","bSortCellsTop","iTabIndex","fnStateLoadCallback","fnStateSaveCallback","renderer",["iCookieDuration","iStateDuration"],["oSearch","oPreviousSearch"],["aoSearchCols","aoPreSearchCols"],["iDisplayLength","_iDisplayLength"],["bJQueryUI","bJUI"]]);D(k.oScroll,h,[["sScrollX","sX"],["sScrollXInner","sXInner"],
+["sScrollY","sY"],["bScrollCollapse","bCollapse"]]);D(k.oLanguage,h,"fnInfoCallback");y(k,"aoDrawCallback",h.fnDrawCallback,"user");y(k,"aoServerParams",h.fnServerParams,"user");y(k,"aoStateSaveParams",h.fnStateSaveParams,"user");y(k,"aoStateLoadParams",h.fnStateLoadParams,"user");y(k,"aoStateLoaded",h.fnStateLoaded,"user");y(k,"aoRowCallback",h.fnRowCallback,"user");y(k,"aoRowCreatedCallback",h.fnCreatedRow,"user");y(k,"aoHeaderCallback",h.fnHeaderCallback,"user");y(k,"aoFooterCallback",h.fnFooterCallback,
+"user");y(k,"aoInitComplete",h.fnInitComplete,"user");y(k,"aoPreDrawCallback",h.fnPreDrawCallback,"user");n=k.oClasses;h.bJQueryUI?(g.extend(n,p.ext.oJUIClasses,h.oClasses),h.sDom===m.sDom&&"lfrtip"===m.sDom&&(k.sDom='<"H"lfr>t<"F"ip>'),k.renderer)?g.isPlainObject(k.renderer)&&!k.renderer.header&&(k.renderer.header="jqueryui"):k.renderer="jqueryui":g.extend(n,p.ext.classes,h.oClasses);g(this).addClass(n.sTable);if(""!==k.oScroll.sX||""!==k.oScroll.sY)k.oScroll.iBarWidth=Db();!0===k.oScroll.sX&&(k.oScroll.sX=
+"100%");k.iInitDisplayStart===l&&(k.iInitDisplayStart=h.iDisplayStart,k._iDisplayStart=h.iDisplayStart);null!==h.iDeferLoading&&(k.bDeferLoading=!0,i=g.isArray(h.iDeferLoading),k._iRecordsDisplay=i?h.iDeferLoading[0]:h.iDeferLoading,k._iRecordsTotal=i?h.iDeferLoading[1]:h.iDeferLoading);""!==h.oLanguage.sUrl?(k.oLanguage.sUrl=h.oLanguage.sUrl,g.getJSON(k.oLanguage.sUrl,null,function(a){M(a);G(m.oLanguage,a);g.extend(true,k.oLanguage,h.oLanguage,a);ra(k)}),e=!0):g.extend(!0,k.oLanguage,h.oLanguage);
+null===h.asStripeClasses&&(k.asStripeClasses=[n.sStripeOdd,n.sStripeEven]);var i=k.asStripeClasses,r=g("tbody tr:eq(0)",this);-1!==g.inArray(!0,g.map(i,function(a){return r.hasClass(a)}))&&(g("tbody tr",this).removeClass(i.join(" ")),k.asDestroyStripes=i.slice());var o=[],q,i=this.getElementsByTagName("thead");0!==i.length&&($(k.aoHeader,i[0]),o=ma(k));if(null===h.aoColumns){q=[];i=0;for(j=o.length;i<j;i++)q.push(null)}else q=h.aoColumns;i=0;for(j=q.length;i<j;i++)Aa(k,o?o[i]:null);gb(k,h.aoColumnDefs,
+q,function(a,b){fa(k,a,b)});if(r.length){var s=function(a,b){return a.getAttribute("data-"+b)?b:null};g.each(ia(k,r[0]).cells,function(a,b){var c=k.aoColumns[a];if(c.mData===a){var d=s(b,"sort")||s(b,"order"),e=s(b,"filter")||s(b,"search");if(d!==null||e!==null){c.mData={_:a+".display",sort:d!==null?a+".@data-"+d:l,type:d!==null?a+".@data-"+d:l,filter:e!==null?a+".@data-"+e:l};fa(k,a)}}})}var u=k.oFeatures;h.bStateSave&&(u.bStateSave=!0,Gb(k,h),y(k,"aoDrawCallback",ta,"state_save"));if(h.aaSorting===
+l){o=k.aaSorting;i=0;for(j=o.length;i<j;i++)o[i][1]=k.aoColumns[i].asSorting[0]}sa(k);u.bSort&&y(k,"aoDrawCallback",function(){if(k.bSorted){var a=Q(k),b={};g.each(a,function(a,c){b[c.src]=c.dir});t(k,null,"order",[k,a,b]);Fb(k)}});y(k,"aoDrawCallback",function(){(k.bSorted||z(k)==="ssp"||u.bDeferRender)&&sa(k)},"sc");eb(k);i=g(this).children("caption").each(function(){this._captionSide=g(this).css("caption-side")});j=g(this).children("thead");0===j.length&&(j=g("<thead/>").appendTo(this));k.nTHead=
+j[0];j=g(this).children("tbody");0===j.length&&(j=g("<tbody/>").appendTo(this));k.nTBody=j[0];j=g(this).children("tfoot");if(0===j.length&&0<i.length&&(""!==k.oScroll.sX||""!==k.oScroll.sY))j=g("<tfoot/>").appendTo(this);0===j.length||0===j.children().length?g(this).addClass(n.sNoFooter):0<j.length&&(k.nTFoot=j[0],$(k.aoFooter,k.nTFoot));if(h.aaData)for(i=0;i<h.aaData.length;i++)H(k,h.aaData[i]);else(k.bDeferLoading||"dom"==z(k))&&ha(k,g(k.nTBody).children("tr"));k.aiDisplay=k.aiDisplayMaster.slice();
+k.bInitialised=!0;!1===e&&ra(k)}});b=null;return this};var Ob=[],v=Array.prototype,Wb=function(a){var b,c,d=p.settings,e=g.map(d,function(a){return a.nTable});if(a){if(a.nTable&&a.oApi)return[a];if(a.nodeName&&"table"===a.nodeName.toLowerCase())return b=g.inArray(a,e),-1!==b?[d[b]]:null;if(a&&"function"===typeof a.settings)return a.settings().toArray();"string"===typeof a?c=g(a):a instanceof g&&(c=a)}else return[];if(c)return c.map(function(){b=g.inArray(this,e);return-1!==b?d[b]:null}).toArray()};
+p.Api=q=function(a,b){if(!this instanceof q)throw"DT API must be constructed as a new object";var c=[],d=function(a){(a=Wb(a))&&c.push.apply(c,a)};if(g.isArray(a))for(var e=0,f=a.length;e<f;e++)d(a[e]);else d(a);this.context=Ja(c);b&&this.push.apply(this,b.toArray?b.toArray():b);this.selector={rows:null,cols:null,opts:null};q.extend(this,this,Ob)};q.prototype={concat:v.concat,context:[],each:function(a){if(v.forEach)v.forEach.call(this,a,this);else for(var b=0,c=this.length;b<c;b++)a.call(this,this[b],
+b,this);return this},eq:function(a){var b=this.context;return b.length>a?new q(b[a],this[a]):null},filter:function(a){var b=[];if(v.filter)b=v.filter.call(this,a,this);else for(var c=0,d=this.length;c<d;c++)a.call(this,this[c],c,this)&&b.push(this[c]);return new q(this.context,b)},flatten:function(){var a=[];return new q(this.context,a.concat.apply(a,this.toArray()))},join:v.join,indexOf:v.indexOf||function(a,b){for(var c=b||0,d=this.length;c<d;c++)if(this[c]===a)return c;return-1},iterator:function(a,
+b,c){var d=[],e,f,h,g,j,n=this.context,m,o,k=this.selector;"string"===typeof a&&(c=b,b=a,a=!1);f=0;for(h=n.length;f<h;f++)if("table"===b)e=c(n[f],f),e!==l&&d.push(e);else if("columns"===b||"rows"===b)e=c(n[f],this[f],f),e!==l&&d.push(e);else if("column"===b||"column-rows"===b||"row"===b||"cell"===b){o=this[f];"column-rows"===b&&(m=Ya(n[f],k.opts));g=0;for(j=o.length;g<j;g++)e=o[g],e="cell"===b?c(n[f],e.row,e.column,f,g):c(n[f],e,f,g,m),e!==l&&d.push(e)}return d.length?(a=new q(n,a?d.concat.apply([],
+d):d),b=a.selector,b.rows=k.rows,b.cols=k.cols,b.opts=k.opts,a):this},lastIndexOf:v.lastIndexOf||function(a,b){return this.indexOf.apply(this.toArray.reverse(),arguments)},length:0,map:function(a){var b=[];if(v.map)b=v.map.call(this,a,this);else for(var c=0,d=this.length;c<d;c++)b.push(a.call(this,this[c],c));return new q(this.context,b)},pluck:function(a){return this.map(function(b){return b[a]})},pop:v.pop,push:v.push,reduce:v.reduce||function(a,b){return fb(this,a,b,0,this.length,1)},reduceRight:v.reduceRight||
+function(a,b){return fb(this,a,b,this.length-1,-1,-1)},reverse:v.reverse,selector:null,shift:v.shift,sort:v.sort,splice:v.splice,toArray:function(){return v.slice.call(this)},to$:function(){return g(this)},toJQuery:function(){return g(this)},unique:function(){return new q(this.context,Ja(this))},unshift:v.unshift};q.extend=function(a,b,c){if(b&&(b instanceof q||b.__dt_wrapper)){var d,e,f,h=function(b,c){return function(){var d=b.apply(a,arguments);q.extend(d,d,c.methodExt);return d}};d=0;for(e=c.length;d<
+e;d++)f=c[d],b[f.name]="function"===typeof f.val?h(f.val,f):g.isPlainObject(f.val)?{}:f.val,b[f.name].__dt_wrapper=!0,q.extend(a,b[f.name],f.propExt)}};q.register=r=function(a,b){if(g.isArray(a))for(var c=0,d=a.length;c<d;c++)q.register(a[c],b);else{for(var e=a.split("."),f=Ob,h,i,c=0,d=e.length;c<d;c++){h=(i=-1!==e[c].indexOf("()"))?e[c].replace("()",""):e[c];var j;a:{j=0;for(var n=f.length;j<n;j++)if(f[j].name===h){j=f[j];break a}j=null}j||(j={name:h,val:{},methodExt:[],propExt:[]},f.push(j));c===
+d-1?j.val=b:f=i?j.methodExt:j.propExt}q.ready&&p.api.build()}};q.registerPlural=x=function(a,b,c){q.register(a,c);q.register(b,function(){var a=c.apply(this,arguments);return a===this?this:a instanceof q?a.length?g.isArray(a[0])?new q(a.context,a[0]):a[0]:l:a})};r("tables()",function(a){var b;if(a){b=q;var c=this.context;if("number"===typeof a)a=[c[a]];else var d=g.map(c,function(a){return a.nTable}),a=g(d).filter(a).map(function(){var a=g.inArray(this,d);return c[a]}).toArray();b=new b(a)}else b=
+this;return b});r("table()",function(a){var a=this.tables(a),b=a.context;return b.length?new q(b[0]):a});x("tables().nodes()","table().node()",function(){return this.iterator("table",function(a){return a.nTable})});x("tables().body()","table().body()",function(){return this.iterator("table",function(a){return a.nTBody})});x("tables().header()","table().header()",function(){return this.iterator("table",function(a){return a.nTHead})});x("tables().footer()","table().footer()",function(){return this.iterator("table",
+function(a){return a.nTFoot})});r("draw()",function(a){return this.iterator("table",function(b){K(b,!1===a)})});r("page()",function(a){return a===l?this.page.info().page:this.iterator("table",function(b){Ra(b,a)})});r("page.info()",function(){if(0===this.context.length)return l;var a=this.context[0],b=a._iDisplayStart,c=a._iDisplayLength,d=a.fnRecordsDisplay(),e=-1===c;return{page:e?0:Math.floor(b/c),pages:e?1:Math.ceil(d/c),start:b,end:a.fnDisplayEnd(),length:c,recordsTotal:a.fnRecordsTotal(),recordsDisplay:d}});
+r("page.len()",function(a){return a===l?0!==this.context.length?this.context[0]._iDisplayLength:l:this.iterator("table",function(b){Pa(b,a)})});var Pb=function(a,b,c){"ssp"==z(a)?K(a,b):(C(a,!0),na(a,[],function(c){ja(a);for(var c=oa(a,c),d=0,h=c.length;d<h;d++)H(a,c[d]);K(a,b);C(a,!1)}));if(c){var d=new q(a);d.one("draw",function(){c(d.ajax.json())})}};r("ajax.json()",function(){var a=this.context;if(0<a.length)return a[0].json});r("ajax.params()",function(){var a=this.context;if(0<a.length)return a[0].oAjaxData});
+r("ajax.reload()",function(a,b){return this.iterator("table",function(c){Pb(c,!1===b,a)})});r("ajax.url()",function(a){var b=this.context;if(a===l){if(0===b.length)return l;b=b[0];return b.ajax?g.isPlainObject(b.ajax)?b.ajax.url:b.ajax:b.sAjaxSource}return this.iterator("table",function(b){g.isPlainObject(b.ajax)?b.ajax.url=a:b.ajax=a})});r("ajax.url().load()",function(a,b){return this.iterator("table",function(c){Pb(c,!1===b,a)})});var Za=function(a,b){var c=[],d,e,f,h,i,j;if(!a||"string"===typeof a||
+a.length===l)a=[a];f=0;for(h=a.length;f<h;f++){e=a[f]&&a[f].split?a[f].split(","):[a[f]];i=0;for(j=e.length;i<j;i++)(d=b("string"===typeof e[i]?g.trim(e[i]):e[i]))&&d.length&&c.push.apply(c,d)}return c},$a=function(a){a||(a={});a.filter&&!a.search&&(a.search=a.filter);return{search:a.search||"none",order:a.order||"current",page:a.page||"all"}},ab=function(a){for(var b=0,c=a.length;b<c;b++)if(0<a[b].length)return a[0]=a[b],a.length=1,a.context=[a.context[b]],a;a.length=0;return a},Ya=function(a,b){var c,
+d,e,f=[],h=a.aiDisplay;c=a.aiDisplayMaster;var i=b.search;d=b.order;e=b.page;if("ssp"==z(a))return"removed"===i?[]:R(0,c.length);if("current"==e){c=a._iDisplayStart;for(d=a.fnDisplayEnd();c<d;c++)f.push(h[c])}else if("current"==d||"applied"==d)f="none"==i?c.slice():"applied"==i?h.slice():g.map(c,function(a){return-1===g.inArray(a,h)?a:null});else if("index"==d||"original"==d){c=0;for(d=a.aoData.length;c<d;c++)"none"==i?f.push(c):(e=g.inArray(c,h),(-1===e&&"removed"==i||1===e&&"applied"==i)&&f.push(c))}return f};
+r("rows()",function(a,b){a===l?a="":g.isPlainObject(a)&&(b=a,a="");var b=$a(b),c=this.iterator("table",function(c){var e=b;return Za(a,function(a){var b=Lb(a);if(b!==null&&!e)return[b];var i=Ya(c,e);if(b!==null&&g.inArray(b,i)!==-1)return[b];if(!a)return i;for(var b=[],j=0,n=i.length;j<n;j++)b.push(c.aoData[i[j]].nTr);return a.nodeName&&g.inArray(a,b)!==-1?[a._DT_RowIndex]:g(b).filter(a).map(function(){return this._DT_RowIndex}).toArray()})});c.selector.rows=a;c.selector.opts=b;return c});r("rows().nodes()",
+function(){return this.iterator("row",function(a,b){return a.aoData[b].nTr||l})});r("rows().data()",function(){return this.iterator(!0,"rows",function(a,b){return xa(a.aoData,b,"_aData")})});x("rows().cache()","row().cache()",function(a){return this.iterator("row",function(b,c){var d=b.aoData[c];return"search"===a?d._aFilterData:d._aSortData})});x("rows().invalidate()","row().invalidate()",function(a){return this.iterator("row",function(b,c){la(b,c,a)})});x("rows().indexes()","row().index()",function(){return this.iterator("row",
+function(a,b){return b})});x("rows().remove()","row().remove()",function(){var a=this;return this.iterator("row",function(b,c,d){var e=b.aoData;e.splice(c,1);for(var f=0,h=e.length;f<h;f++)null!==e[f].nTr&&(e[f].nTr._DT_RowIndex=f);g.inArray(c,b.aiDisplay);ka(b.aiDisplayMaster,c);ka(b.aiDisplay,c);ka(a[d],c,!1);Qa(b)})});r("rows.add()",function(a){var b=this.iterator("table",function(b){var c,f,h,g=[];f=0;for(h=a.length;f<h;f++)c=a[f],c.nodeName&&"TR"===c.nodeName.toUpperCase()?g.push(ha(b,c)[0]):
+g.push(H(b,c));return g}),c=this.rows(-1);c.pop();c.push.apply(c,b.toArray());return c});r("row()",function(a,b){return ab(this.rows(a,b))});r("row().data()",function(a){var b=this.context;if(a===l)return b.length&&this.length?b[0].aoData[this[0]]._aData:l;b[0].aoData[this[0]]._aData=a;la(b[0],this[0],"data");return this});r("row().node()",function(){var a=this.context;return a.length&&this.length?a[0].aoData[this[0]].nTr||null:null});r("row.add()",function(a){a instanceof g&&a.length&&(a=a[0]);var b=
+this.iterator("table",function(b){return a.nodeName&&"TR"===a.nodeName.toUpperCase()?ha(b,a)[0]:H(b,a)});return this.row(b[0])});var Qb=function(a){var b=this.context;if(b.length&&this.length){var c=b[0].aoData[this[0]];if(c._details){(c._detailsShow=a)?c._details.insertAfter(c.nTr):c._details.remove();var d=b[0],e=new q(d);e.off("draw.dt.DT_details column-visibility.dt.DT_details");0<B(d.aoData,"_details").length&&(e.on("draw.dt.DT_details",function(){e.rows({page:"current"}).eq(0).each(function(a){a=
+d.aoData[a];a._detailsShow&&a._details.insertAfter(a.nTr)})}),e.on("column-visibility.dt.DT_details",function(a,b){for(var c,d=Y(b),e=0,g=b.aoData.length;e<g;e++)c=b.aoData[e],c._details&&c._details.children("td[colspan]").attr("colspan",d)}))}}return this};r("row().child()",function(a,b){var c=this.context;if(a===l)return c.length&&this.length?c[0].aoData[this[0]]._details:l;if(c.length&&this.length){var d=c[0],c=c[0].aoData[this[0]],e=[],f=function(a,b){if(a.nodeName&&"tr"===a.nodeName.toLowerCase())e.push(a);
+else{var c=g("<tr><td/></tr>");g("td",c).addClass(b).html(a)[0].colSpan=Y(d);e.push(c[0])}};if(g.isArray(a)||a instanceof g)for(var h=0,i=a.length;h<i;h++)f(a[h],b);else f(a,b);c._details&&c._details.remove();c._details=g(e);c._detailsShow&&c._details.insertAfter(c.nTr)}return this});r(["row().child.show()","row().child().show()"],function(){Qb.call(this,!0);return this});r(["row().child.hide()","row().child().hide()"],function(){Qb.call(this,!1);return this});r("row().child.isShown()",function(){var a=
+this.context;return a.length&&this.length?a[0].aoData[this[0]]._detailsShow||!1:!1});var Xb=/^(.*):(name|visIdx|visible)$/;r("columns()",function(a,b){a===l?a="":g.isPlainObject(a)&&(b=a,a="");var b=$a(b),c=this.iterator("table",function(b){var c=a,f=b.aoColumns,h=B(f,"sName"),i=B(f,"nTh");return Za(c,function(a){var c=Lb(a);if(a==="")return R(f.length);if(c!==null)return[c>=0?c:f.length+c];var e=typeof a==="string"?a.match(Xb):"";if(e)switch(e[2]){case "visIdx":case "visible":a=parseInt(e[1],10);
+if(a<0){c=g.map(f,function(a,b){return a.bVisible?b:null});return[c[c.length+a]]}return[ga(b,a)];case "name":return g.map(h,function(a,b){return a===e[1]?b:null})}else return g(i).filter(a).map(function(){return g.inArray(this,i)}).toArray()})});c.selector.cols=a;c.selector.opts=b;return c});x("columns().header()","column().header()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTh})});x("columns().footer()","column().footer()",function(){return this.iterator("column",
+function(a,b){return a.aoColumns[b].nTf})});x("columns().data()","column().data()",function(){return this.iterator("column-rows",function(a,b,c,d,e){for(var c=[],d=0,f=e.length;d<f;d++)c.push(A(a,e[d],b,""));return c})});x("columns().cache()","column().cache()",function(a){return this.iterator("column-rows",function(b,c,d,e,f){return xa(b.aoData,f,"search"===a?"_aFilterData":"_aSortData",c)})});x("columns().nodes()","column().nodes()",function(){return this.iterator("column-rows",function(a,b,c,d,
+e){return xa(a.aoData,e,"anCells",b)})});x("columns().visible()","column().visible()",function(a){return this.iterator("column",function(b,c){var d;if(a===l)d=b.aoColumns[c].bVisible;else{var e=b.aoColumns;d=e[c];var f=b.aoData,h,i,j;if(a===l)d=d.bVisible;else{if(d.bVisible!==a){if(a){var n=g.inArray(!0,B(e,"bVisible"),c+1);h=0;for(i=f.length;h<i;h++)j=f[h].nTr,e=f[h].anCells,j&&j.insertBefore(e[c],e[n]||null)}else g(B(b.aoData,"anCells",c)).detach(),d.bVisible=!1,I(b,b.aoHeader),I(b,b.aoFooter),
+ta(b);d.bVisible=a;I(b,b.aoHeader);I(b,b.aoFooter);U(b);(b.oScroll.sX||b.oScroll.sY)&&V(b);t(b,null,"column-visibility",[b,c,a]);ta(b)}d=void 0}}return d})});x("columns().indexes()","column().index()",function(a){return this.iterator("column",function(b,c){return"visible"===a?X(b,c):c})});r("columns.adjust()",function(){return this.iterator("table",function(a){U(a)})});r("column.index()",function(a,b){if(0!==this.context.length){var c=this.context[0];if("fromVisible"===a||"toData"===a)return ga(c,
+b);if("fromData"===a||"toVisible"===a)return X(c,b)}});r("column()",function(a,b){return ab(this.columns(a,b))});r("cells()",function(a,b,c){g.isPlainObject(a)&&(a.row?(c=b,b=null):(c=a,a=null));g.isPlainObject(b)&&(c=b,b=null);if(null===b||b===l)return this.iterator("table",function(b){var d=a,e=$a(c),f=b.aoData,h=Ya(b,e),e=xa(f,h,"anCells"),i=g([].concat.apply([],e)),j,m=b.aoColumns.length,n,l,p,r;return Za(d,function(a){if(a){if(g.isPlainObject(a))return[a]}else{n=[];l=0;for(p=h.length;l<p;l++){j=
+h[l];for(r=0;r<m;r++)n.push({row:j,column:r})}return n}return i.filter(a).map(function(a,b){j=b.parentNode._DT_RowIndex;return{row:j,column:g.inArray(b,f[j].anCells)}}).toArray()})});var d=this.columns(b,c),e=this.rows(a,c),f,h,i,j,n,m=this.iterator("table",function(a,b){f=[];h=0;for(i=e[b].length;h<i;h++){j=0;for(n=d[b].length;j<n;j++)f.push({row:e[b][h],column:d[b][j]})}return f});g.extend(m.selector,{cols:b,rows:a,opts:c});return m});x("cells().nodes()","cell().node()",function(){return this.iterator("cell",
+function(a,b,c){return a.aoData[b].anCells[c]})});r("cells().data()",function(){return this.iterator("cell",function(a,b,c){return A(a,b,c)})});x("cells().cache()","cell().cache()",function(a){a="search"===a?"_aFilterData":"_aSortData";return this.iterator("cell",function(b,c,d){return b.aoData[c][a][d]})});x("cells().indexes()","cell().index()",function(){return this.iterator("cell",function(a,b,c){return{row:b,column:c,columnVisible:X(a,c)}})});r(["cells().invalidate()","cell().invalidate()"],function(a){var b=
+this.selector;this.rows(b.rows,b.opts).invalidate(a);return this});r("cell()",function(a,b,c){return ab(this.cells(a,b,c))});r("cell().data()",function(a){var b=this.context,c=this[0];if(a===l)return b.length&&c.length?A(b[0],c[0].row,c[0].column):l;Ea(b[0],c[0].row,c[0].column,a);la(b[0],c[0].row,"data",c[0].column);return this});r("order()",function(a,b){var c=this.context;if(a===l)return 0!==c.length?c[0].aaSorting:l;"number"===typeof a?a=[[a,b]]:g.isArray(a[0])||(a=Array.prototype.slice.call(arguments));
+return this.iterator("table",function(b){b.aaSorting=a.slice()})});r("order.listener()",function(a,b,c){return this.iterator("table",function(d){Ka(d,a,b,c)})});r(["columns().order()","column().order()"],function(a){var b=this;return this.iterator("table",function(c,d){var e=[];g.each(b[d],function(b,c){e.push([c,a])});c.aaSorting=e})});r("search()",function(a,b,c,d){var e=this.context;return a===l?0!==e.length?e[0].oPreviousSearch.sSearch:l:this.iterator("table",function(e){e.oFeatures.bFilter&&
+aa(e,g.extend({},e.oPreviousSearch,{sSearch:a+"",bRegex:null===b?!1:b,bSmart:null===c?!0:c,bCaseInsensitive:null===d?!0:d}),1)})});r(["columns().search()","column().search()"],function(a,b,c,d){return this.iterator("column",function(e,f){var h=e.aoPreSearchCols;if(a===l)return h[f].sSearch;e.oFeatures.bFilter&&(g.extend(h[f],{sSearch:a+"",bRegex:null===b?!1:b,bSmart:null===c?!0:c,bCaseInsensitive:null===d?!0:d}),aa(e,e.oPreviousSearch,1))})});p.versionCheck=p.fnVersionCheck=function(a){for(var b=
+p.version.split("."),a=a.split("."),c,d,e=0,f=a.length;e<f;e++)if(c=parseInt(b[e],10)||0,d=parseInt(a[e],10)||0,c!==d)return c>d;return!0};p.isDataTable=p.fnIsDataTable=function(a){var b=g(a).get(0),c=!1;g.each(p.settings,function(a,e){if(e.nTable===b||e.nScrollHead===b||e.nScrollFoot===b)c=!0});return c};p.tables=p.fnTables=function(a){return jQuery.map(p.settings,function(b){if(!a||a&&g(b.nTable).is(":visible"))return b.nTable})};p.camelToHungarian=G;r("$()",function(a,b){var c=this.rows(b).nodes(),
+c=g(c);return g([].concat(c.filter(a).toArray(),c.find(a).toArray()))});g.each(["on","one","off"],function(a,b){r(b+"()",function(){var a=Array.prototype.slice.call(arguments);-1===a[0].indexOf(".dt")&&(a[0]+=".dt");var d=g(this.tables().nodes());d[b].apply(d,a);return this})});r("clear()",function(){return this.iterator("table",function(a){ja(a)})});r("settings()",function(){return new q(this.context,this.context)});r("data()",function(){return this.iterator("table",function(a){return B(a.aoData,
+"_aData")}).flatten()});r("destroy()",function(a){a=a||!1;return this.iterator("table",function(b){var c=b.nTableWrapper.parentNode,d=b.oClasses,e=b.nTable,f=b.nTBody,h=b.nTHead,i=b.nTFoot,j=g(e),f=g(f),n=g(b.nTableWrapper),m=g.map(b.aoData,function(a){return a.nTr}),l;b.bDestroying=!0;t(b,"aoDestroyCallback","destroy",[b]);a||(new q(b)).columns().visible(!0);n.unbind(".DT").find(":not(tbody *)").unbind(".DT");g(za).unbind(".DT-"+b.sInstance);e!=h.parentNode&&(j.children("thead").detach(),j.append(h));
+i&&e!=i.parentNode&&(j.children("tfoot").detach(),j.append(i));j.detach();n.detach();b.aaSorting=[];b.aaSortingFixed=[];sa(b);g(m).removeClass(b.asStripeClasses.join(" "));g("th, td",h).removeClass(d.sSortable+" "+d.sSortableAsc+" "+d.sSortableDesc+" "+d.sSortableNone);b.bJUI&&(g("th span."+d.sSortIcon+", td span."+d.sSortIcon,h).detach(),g("th, td",h).each(function(){var a=g("div."+d.sSortJUIWrapper,this);g(this).append(a.contents());a.detach()}));!a&&c&&c.insertBefore(e,b.nTableReinsertBefore);
+f.children().detach();f.append(m);j.css("width",b.sDestroyWidth).removeClass(d.sTable);(l=b.asDestroyStripes.length)&&f.children().each(function(a){g(this).addClass(b.asDestroyStripes[a%l])});c=g.inArray(b,p.settings);-1!==c&&p.settings.splice(c,1)})});p.version="1.10.0";p.settings=[];p.models={};p.models.oSearch={bCaseInsensitive:!0,sSearch:"",bRegex:!1,bSmart:!0};p.models.oRow={nTr:null,anCells:null,_aData:[],_aSortData:null,_aFilterData:null,_sFilterRow:null,_sRowStripe:"",src:null};p.models.oColumn=
+{idx:null,aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bVisible:null,_sManualType:null,_bAttrSrc:!1,fnCreatedCell:null,fnGetData:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null,sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};p.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:[],ajax:null,aLengthMenu:[10,25,50,100],aoColumns:null,
+aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:!0,bDeferRender:!1,bDestroy:!1,bFilter:!0,bInfo:!0,bJQueryUI:!1,bLengthChange:!0,bPaginate:!0,bProcessing:!1,bRetrieve:!1,bScrollCollapse:!1,bServerSide:!1,bSort:!0,bSortMulti:!0,bSortCellsTop:!1,bSortClasses:!0,bStateSave:!1,fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(a){return a.toString().replace(/\B(?=(\d{3})+(?!\d))/g,this.oLanguage.sThousands)},fnHeaderCallback:null,fnInfoCallback:null,
+fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:null,fnServerParams:null,fnStateLoadCallback:function(a){try{return JSON.parse((-1===a.iStateDuration?sessionStorage:localStorage).getItem("DataTables_"+a.sInstance+"_"+location.pathname))}catch(b){}},fnStateLoadParams:null,fnStateLoaded:null,fnStateSaveCallback:function(a,b){try{(-1===a.iStateDuration?sessionStorage:localStorage).setItem("DataTables_"+a.sInstance+"_"+location.pathname,JSON.stringify(b))}catch(c){}},fnStateSaveParams:null,
+iStateDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iTabIndex:0,oClasses:{},oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"},oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",sInfoPostFix:"",sDecimal:"",
+sThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sUrl:"",sZeroRecords:"No matching records found"},oSearch:g.extend({},p.models.oSearch),sAjaxDataProp:"data",sAjaxSource:null,sDom:"lfrtip",sPaginationType:"simple_numbers",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET",renderer:null};S(p.defaults);p.defaults.column={aDataSort:null,iDataSort:-1,asSorting:["asc","desc"],bSearchable:!0,bSortable:!0,bVisible:!0,fnCreatedCell:null,
+mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null};S(p.defaults.column);p.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortMulti:null,bSortClasses:null,bStateSave:null},oScroll:{bCollapse:null,iBarWidth:0,sX:null,sXInner:null,sY:null},oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:!1,
+bScrollbarLeft:!1},ajax:null,aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aoColumns:[],aoHeader:[],aoFooter:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:[],asStripeClasses:null,asDestroyStripes:[],sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],aoStateLoaded:[],sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,
+nTableWrapper:null,bDeferLoading:!1,bInitialised:!1,aoOpenRows:[],sDom:null,sPaginationType:"two_button",iStateDuration:0,aoStateSave:[],aoStateLoad:[],oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,bAjaxDataGet:!0,jqXHR:null,json:l,oAjaxData:l,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:!1,iDrawError:-1,_iDisplayLength:10,_iDisplayStart:0,_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:!1,bSorted:!1,bSortCellsTop:null,
+oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return"ssp"==z(this)?1*this._iRecordsTotal:this.aiDisplayMaster.length},fnRecordsDisplay:function(){return"ssp"==z(this)?1*this._iRecordsDisplay:this.aiDisplay.length},fnDisplayEnd:function(){var a=this._iDisplayLength,b=this._iDisplayStart,c=b+a,d=this.aiDisplay.length,e=this.oFeatures,f=e.bPaginate;return e.bServerSide?!1===f||-1===a?b+d:Math.min(b+a,this._iRecordsDisplay):!f||c>d||-1===a?d:c},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,
+nScrollFoot:null,aLastSort:[],oPlugins:{}};p.ext=u={classes:{},errMode:"alert",feature:[],search:[],internal:{},legacy:{ajax:null},pager:{},renderer:{pageButton:{},header:{}},order:{},type:{detect:[],search:{},order:{}},_unique:0,fnVersionCheck:p.fnVersionCheck,iApiIndex:0,oJUIClasses:{},sVersion:p.version};g.extend(u,{afnFiltering:u.search,aTypes:u.type.detect,ofnSearch:u.type.search,oSort:u.type.order,afnSortData:u.order,aoFeatures:u.feature,oApi:u.internal,oStdClasses:u.classes,oPagination:u.pager});
+g.extend(p.ext.classes,{sTable:"dataTable",sNoFooter:"no-footer",sPageButton:"paginate_button",sPageButtonActive:"current",sPageButtonDisabled:"disabled",sStripeOdd:"odd",sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing",sSortAsc:"sorting_asc",sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled",
+sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sFilterInput:"",sLengthSelect:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",sHeaderTH:"",sFooterTH:"",sSortJUIAsc:"",sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"",
+sJUIHeader:"",sJUIFooter:""});var ya="",ya="",E=ya+"ui-state-default",ea=ya+"css_right ui-icon ui-icon-",Rb=ya+"fg-toolbar ui-toolbar ui-widget-header ui-helper-clearfix";g.extend(p.ext.oJUIClasses,p.ext.classes,{sPageButton:"fg-button ui-button "+E,sPageButtonActive:"ui-state-disabled",sPageButtonDisabled:"ui-state-disabled",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_",sSortAsc:E+" sorting_asc",sSortDesc:E+" sorting_desc",sSortable:E+" sorting",
+sSortableAsc:E+" sorting_asc_disabled",sSortableDesc:E+" sorting_desc_disabled",sSortableNone:E+" sorting_disabled",sSortJUIAsc:ea+"triangle-1-n",sSortJUIDesc:ea+"triangle-1-s",sSortJUI:ea+"carat-2-n-s",sSortJUIAscAllowed:ea+"carat-1-n",sSortJUIDescAllowed:ea+"carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead "+E,sScrollFoot:"dataTables_scrollFoot "+E,sHeaderTH:E,sFooterTH:E,sJUIHeader:Rb+" ui-corner-tl ui-corner-tr",sJUIFooter:Rb+
+" ui-corner-bl ui-corner-br"});var Ib=p.ext.pager;g.extend(Ib,{simple:function(){return["previous","next"]},full:function(){return["first","previous","next","last"]},simple_numbers:function(a,b){return["previous",Ua(a,b),"next"]},full_numbers:function(a,b){return["first","previous",Ua(a,b),"next","last"]},_numbers:Ua,numbers_length:7});g.extend(!0,p.ext.renderer,{pageButton:{_:function(a,b,c,d,e,f){var h=a.oClasses,i=a.oLanguage.oPaginate,j,l,m=0,o=function(b,d){var k,p,r,q,s=function(b){Ra(a,b.data.action,
+true)};k=0;for(p=d.length;k<p;k++){q=d[k];if(g.isArray(q)){r=g("<"+(q.DT_el||"div")+"/>").appendTo(b);o(r,q)}else{l=j="";switch(q){case "ellipsis":b.append("<span>&hellip;</span>");break;case "first":j=i.sFirst;l=q+(e>0?"":" "+h.sPageButtonDisabled);break;case "previous":j=i.sPrevious;l=q+(e>0?"":" "+h.sPageButtonDisabled);break;case "next":j=i.sNext;l=q+(e<f-1?"":" "+h.sPageButtonDisabled);break;case "last":j=i.sLast;l=q+(e<f-1?"":" "+h.sPageButtonDisabled);break;default:j=q+1;l=e===q?h.sPageButtonActive:
+""}if(j){r=g("<a>",{"class":h.sPageButton+" "+l,"aria-controls":a.sTableId,"data-dt-idx":m,tabindex:a.iTabIndex,id:c===0&&typeof q==="string"?a.sTableId+"_"+q:null}).html(j).appendTo(b);Ta(r,{action:q},s);m++}}}},k=g(N.activeElement).data("dt-idx");o(g(b).empty(),d);k!==null&&g(b).find("[data-dt-idx="+k+"]").focus()}}});var va=function(a,b,c,d){if(!a||"-"===a)return-Infinity;b&&(a=Mb(a,b));a.replace&&(c&&(a=a.replace(c,"")),d&&(a=a.replace(d,"")));return 1*a};g.extend(u.type.order,{"date-pre":function(a){return Date.parse(a)||
+0},"html-pre":function(a){return!a?"":a.replace?a.replace(/<.*?>/g,"").toLowerCase():a+""},"string-pre":function(a){return"string"===typeof a?a.toLowerCase():!a||!a.toString?"":a.toString()},"string-asc":function(a,b){return a<b?-1:a>b?1:0},"string-desc":function(a,b){return a<b?1:a>b?-1:0}});bb("");g.extend(p.ext.type.detect,[function(a,b){var c=b.oLanguage.sDecimal;return Xa(a,c)?"num"+c:null},function(a){if(a&&!Vb.test(a))return null;var b=Date.parse(a);return null!==b&&!isNaN(b)||da(a)?"date":
+null},function(a,b){var c=b.oLanguage.sDecimal;return Xa(a,c,!0)?"num-fmt"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Nb(a,c)?"html-num"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Nb(a,c,!0)?"html-num-fmt"+c:null},function(a){return da(a)||"string"===typeof a&&-1!==a.indexOf("<")?"html":null}]);g.extend(p.ext.type.search,{html:function(a){return da(a)?"":"string"===typeof a?a.replace(Kb," ").replace(wa,""):""},string:function(a){return da(a)?"":"string"===typeof a?a.replace(Kb,
+" "):a}});g.extend(!0,p.ext.renderer,{header:{_:function(a,b,c,d){g(a.nTable).on("order.dt.DT",function(a,f,g,i){a=c.idx;b.removeClass(c.sSortingClass+" "+d.sSortAsc+" "+d.sSortDesc).addClass(i[a]=="asc"?d.sSortAsc:i[a]=="desc"?d.sSortDesc:c.sSortingClass)})},jqueryui:function(a,b,c,d){var e=c.idx;g("<div/>").addClass(d.sSortJUIWrapper).append(b.contents()).append(g("<span/>").addClass(d.sSortIcon+" "+c.sSortingClassJUI)).appendTo(b);g(a.nTable).on("order.dt.DT",function(a,g,i,j){b.removeClass(d.sSortAsc+
+" "+d.sSortDesc).addClass(j[e]=="asc"?d.sSortAsc:j[e]=="desc"?d.sSortDesc:c.sSortingClass);b.find("span."+d.sSortIcon).removeClass(d.sSortJUIAsc+" "+d.sSortJUIDesc+" "+d.sSortJUI+" "+d.sSortJUIAscAllowed+" "+d.sSortJUIDescAllowed).addClass(j[e]=="asc"?d.sSortJUIAsc:j[e]=="desc"?d.sSortJUIDesc:c.sSortingClassJUI)})}}});p.render={number:function(a,b,c,d){return{display:function(e){var e=parseFloat(e),f=parseInt(e,10),e=c?(b+(e-f).toFixed(c)).substring(2):"";return(d||"")+f.toString().replace(/\B(?=(\d{3})+(?!\d))/g,
+a)+e}}}};g.extend(p.ext.internal,{_fnExternApiFunc:Jb,_fnBuildAjax:na,_fnAjaxUpdate:ib,_fnAjaxParameters:rb,_fnAjaxUpdateDraw:sb,_fnAjaxDataSrc:oa,_fnAddColumn:Aa,_fnColumnOptions:fa,_fnAdjustColumnSizing:U,_fnVisibleToColumnIndex:ga,_fnColumnIndexToVisible:X,_fnVisbleColumns:Y,_fnGetColumns:W,_fnColumnTypes:Da,_fnApplyColumnDefs:gb,_fnHungarianMap:S,_fnCamelToHungarian:G,_fnLanguageCompat:M,_fnBrowserDetect:eb,_fnAddData:H,_fnAddTr:ha,_fnNodeToDataIndex:function(a,b){return b._DT_RowIndex!==l?b._DT_RowIndex:
+null},_fnNodeToColumnIndex:function(a,b,c){return g.inArray(c,a.aoData[b].anCells)},_fnGetCellData:A,_fnSetCellData:Ea,_fnSplitObjNotation:Ga,_fnGetObjectDataFn:T,_fnSetObjectDataFn:Ba,_fnGetDataMaster:Ha,_fnClearTable:ja,_fnDeleteIndex:ka,_fnInvalidateRow:la,_fnGetRowElements:ia,_fnCreateTr:Fa,_fnBuildHead:hb,_fnDrawHead:I,_fnDraw:J,_fnReDraw:K,_fnAddOptionsHtml:kb,_fnDetectHeader:$,_fnGetUniqueThs:ma,_fnFeatureHtmlFilter:mb,_fnFilterComplete:aa,_fnFilterCustom:vb,_fnFilterColumn:ub,_fnFilter:tb,
+_fnFilterCreateSearch:Na,_fnEscapeRegex:Oa,_fnFilterData:wb,_fnFeatureHtmlInfo:pb,_fnUpdateInfo:xb,_fnInfoMacros:yb,_fnInitialise:ra,_fnInitComplete:pa,_fnLengthChange:Pa,_fnFeatureHtmlLength:lb,_fnFeatureHtmlPaginate:qb,_fnPageChange:Ra,_fnFeatureHtmlProcessing:nb,_fnProcessingDisplay:C,_fnFeatureHtmlTable:ob,_fnScrollDraw:V,_fnApplyToChildren:F,_fnCalculateColumnWidths:Ca,_fnThrottle:Ma,_fnConvertToWidth:zb,_fnScrollingWidthAdjust:Bb,_fnGetWidestNode:Ab,_fnGetMaxLenString:Cb,_fnStringToCss:s,_fnScrollBarWidth:Db,
+_fnSortFlatten:Q,_fnSort:jb,_fnSortAria:Fb,_fnSortListener:Sa,_fnSortAttachListener:Ka,_fnSortingClasses:sa,_fnSortData:Eb,_fnSaveState:ta,_fnLoadState:Gb,_fnSettingsFromNode:ua,_fnLog:O,_fnMap:D,_fnBindAction:Ta,_fnCallbackReg:y,_fnCallbackFire:t,_fnLengthOverflow:Qa,_fnRenderer:La,_fnDataSource:z,_fnRowAttributes:Ia,_fnCalculateEnd:function(){}});g.fn.dataTable=p;g.fn.dataTableSettings=p.settings;g.fn.dataTableExt=p.ext;g.fn.DataTable=function(a){return g(this).dataTable(a).api()};g.each(p,function(a,
+b){g.fn.DataTable[a]=b});return g.fn.dataTable};"function"===typeof define&&define.amd?define("datatables",["jquery"],M):"object"===typeof exports?M(require("jquery")):jQuery&&!jQuery.fn.dataTable&&M(jQuery)})(window,document);
\ No newline at end of file


[04/13] drill git commit: DRILL-5663: Drillbit fails to start when only keystore path is provided without keystore password.

Posted by jn...@apache.org.
DRILL-5663: Drillbit fails to start when only keystore path is provided without keystore password.

Added comments for the drill-override-example.conf file

close #874


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/f1e1dfe0
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/f1e1dfe0
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/f1e1dfe0

Branch: refs/heads/master
Commit: f1e1dfe09c3e57bdeb313d01940275b3fc83535f
Parents: 109b2c0
Author: Sindhuri Rayavaram <si...@srayava-E422.local>
Authored: Thu Jul 13 16:07:53 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 .../src/resources/drill-override-example.conf   |  19 ++--
 .../org/apache/drill/exec/ExecConstants.java    |   8 +-
 .../java/org/apache/drill/exec/SSLConfig.java   |  69 +++++++++++++
 .../drill/exec/server/rest/WebServer.java       |  30 +++---
 .../src/main/resources/drill-module.conf        |  14 ++-
 .../org/apache/drill/exec/TestSSLConfig.java    | 100 +++++++++++++++++++
 6 files changed, 216 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/distribution/src/resources/drill-override-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index 8010f85..0b66f68 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -93,6 +93,17 @@ drill.exec: {
       credentials: true
     }
   },
+  # Below SSL parameters need to be set for custom transport layer settings.
+  ssl: {
+    #If not provided then the default value is java system property javax.net.ssl.keyStore value
+    keyStorePath: "/keystore.file",
+    #If not provided then the default value is java system property javax.net.ssl.keyStorePassword value
+    keyStorePassword: "ks_passwd",
+    #If not provided then the default value is java system property javax.net.ssl.trustStore value
+    trustStorePath: "/truststore.file",
+    #If not provided then the default value is java system property javax.net.ssl.trustStorePassword value
+    trustStorePassword: "ts_passwd"
+  },
   functions: ["org.apache.drill.expr.fn.impl"],
   network: {
     start: 35000
@@ -213,11 +224,5 @@ drill.exec: {
   default_temporary_workspace: "dfs.tmp"
 }
 
-# Below SSL parameters need to be set for custom transport layer settings.
-javax.net.ssl {
-  keyStore: "/keystore.file",
-  keyStorePassword: "ks_passwd",
-  trustStore: "/truststore.file",
-  trustStorePassword: "ts_passwd"
-}
+
 

http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 97cb321..a88875f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -122,10 +122,10 @@ public interface ExecConstants {
   String HTTP_SESSION_MEMORY_RESERVATION = "drill.exec.http.session.memory.reservation";
   String HTTP_SESSION_MEMORY_MAXIMUM = "drill.exec.http.session.memory.maximum";
   String HTTP_SESSION_MAX_IDLE_SECS = "drill.exec.http.session_max_idle_secs";
-  String HTTP_KEYSTORE_PATH = "javax.net.ssl.keyStore";
-  String HTTP_KEYSTORE_PASSWORD = "javax.net.ssl.keyStorePassword";
-  String HTTP_TRUSTSTORE_PATH = "javax.net.ssl.trustStore";
-  String HTTP_TRUSTSTORE_PASSWORD = "javax.net.ssl.trustStorePassword";
+  String HTTP_KEYSTORE_PATH = "drill.exec.ssl.keyStorePath";
+  String HTTP_KEYSTORE_PASSWORD = "drill.exec.ssl.keyStorePassword";
+  String HTTP_TRUSTSTORE_PATH = "drill.exec.ssl.trustStorePath";
+  String HTTP_TRUSTSTORE_PASSWORD = "drill.exec.ssl.trustStorePassword";
   String SYS_STORE_PROVIDER_CLASS = "drill.exec.sys.store.provider.class";
   String SYS_STORE_PROVIDER_LOCAL_PATH = "drill.exec.sys.store.provider.local.path";
   String SYS_STORE_PROVIDER_LOCAL_ENABLE_WRITE = "drill.exec.sys.store.provider.local.write";

http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/exec/java-exec/src/main/java/org/apache/drill/exec/SSLConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/SSLConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/SSLConfig.java
new file mode 100644
index 0000000..c6d6374
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/SSLConfig.java
@@ -0,0 +1,69 @@
+/*
+ * 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.drill.exec;
+
+import com.typesafe.config.Config;
+import org.apache.drill.common.exceptions.DrillException;
+
+public class SSLConfig {
+
+  private final String keystorePath;
+
+  private final String keystorePassword;
+
+  private final String truststorePath;
+
+  private final String truststorePassword;
+
+
+  public SSLConfig(Config config) throws DrillException {
+
+    keystorePath = config.getString(ExecConstants.HTTP_KEYSTORE_PATH).trim();
+
+    keystorePassword = config.getString(ExecConstants.HTTP_KEYSTORE_PASSWORD).trim();
+
+    truststorePath = config.getString(ExecConstants.HTTP_TRUSTSTORE_PATH).trim();
+
+    truststorePassword = config.getString(ExecConstants.HTTP_TRUSTSTORE_PASSWORD).trim();
+
+    /*If keystorePath or keystorePassword is provided in the configuration file use that*/
+    if (!keystorePath.isEmpty() || !keystorePassword.isEmpty()) {
+      if (keystorePath.isEmpty()) {
+        throw new DrillException(" *.ssl.keyStorePath in the configuration file is empty, but *.ssl.keyStorePassword is set");
+      }
+      else if (keystorePassword.isEmpty()){
+        throw new DrillException(" *.ssl.keyStorePassword in the configuration file is empty, but *.ssl.keyStorePath is set ");
+      }
+
+    }
+  }
+
+  public boolean isSslValid() {  return !keystorePath.isEmpty() && !keystorePassword.isEmpty(); }
+
+  public String getKeyStorePath() {  return keystorePath; }
+
+  public String getKeyStorePassword() {  return keystorePassword; }
+
+  public boolean hasTrustStorePath() {  return !truststorePath.isEmpty(); }
+
+  public boolean hasTrustStorePassword() {  return !truststorePassword.isEmpty(); }
+
+  public String getTrustStorePath() {  return truststorePath; }
+
+  public String getTrustStorePassword() {  return truststorePassword; }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index b3fb692..1706b71 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -24,8 +24,11 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.common.exceptions.DrillException;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.SSLConfig;
+import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.rest.auth.DrillRestLoginService;
@@ -139,7 +142,12 @@ public class WebServer implements AutoCloseable {
 
     final ServerConnector serverConnector;
     if (config.getBoolean(ExecConstants.HTTP_ENABLE_SSL)) {
-      serverConnector = createHttpsConnector();
+      try {
+        serverConnector = createHttpsConnector();
+      }
+      catch(DrillException e){
+        throw new DrillbitStartupException(e.getMessage(), e);
+      }
     } else {
       serverConnector = createHttpConnector();
     }
@@ -263,18 +271,16 @@ public class WebServer implements AutoCloseable {
     logger.info("Setting up HTTPS connector for web server");
 
     final SslContextFactory sslContextFactory = new SslContextFactory();
-
-    if (config.hasPath(ExecConstants.HTTP_KEYSTORE_PATH) &&
-        !Strings.isNullOrEmpty(config.getString(ExecConstants.HTTP_KEYSTORE_PATH))) {
+    SSLConfig ssl = new SSLConfig(config);
+    if(ssl.isSslValid()){
       logger.info("Using configured SSL settings for web server");
-      sslContextFactory.setKeyStorePath(config.getString(ExecConstants.HTTP_KEYSTORE_PATH));
-      sslContextFactory.setKeyStorePassword(config.getString(ExecConstants.HTTP_KEYSTORE_PASSWORD));
-
-      // TrustStore and TrustStore password are optional
-      if (config.hasPath(ExecConstants.HTTP_TRUSTSTORE_PATH)) {
-        sslContextFactory.setTrustStorePath(config.getString(ExecConstants.HTTP_TRUSTSTORE_PATH));
-        if (config.hasPath(ExecConstants.HTTP_TRUSTSTORE_PASSWORD)) {
-          sslContextFactory.setTrustStorePassword(config.getString(ExecConstants.HTTP_TRUSTSTORE_PASSWORD));
+
+      sslContextFactory.setKeyStorePath(ssl.getKeyStorePath());
+      sslContextFactory.setKeyStorePassword(ssl.getKeyStorePassword());
+      if(ssl.hasTrustStorePath()){
+        sslContextFactory.setTrustStorePath(ssl.getTrustStorePath());
+        if(ssl.hasTrustStorePassword()){
+          sslContextFactory.setTrustStorePassword(ssl.getTrustStorePassword());
         }
       }
     } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 146df1f..437862e 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -51,7 +51,12 @@ drill.client: {
 // By default ${DRILL_TMP_DIR} is used if set or ${drill.tmp-dir} if it's been overridden.
 drill.tmp-dir: "/tmp"
 drill.tmp-dir: ${?DRILL_TMP_DIR}
-
+javax.net.ssl: {
+  keyStore = "",
+  keyStorePassword = "",
+  trustStore = "",
+  trustStorePassword = ""
+},
 drill.exec: {
   cluster-id: "drillbits1"
   rpc: {
@@ -134,6 +139,13 @@ drill.exec: {
         }
     }
   },
+  //setting javax variables for ssl configurations is being deprecated.
+  ssl: {
+    keyStorePath = ${?javax.net.ssl.keyStore}
+    keyStorePassword = ${?javax.net.ssl.keyStorePassword}
+    trustStorePath = ${?javax.net.ssl.trustStore}
+    trustStorePassword =  ${?javax.net.ssl.trustStorePassword}
+  },
   network: {
     start: 35000
   },

http://git-wip-us.apache.org/repos/asf/drill/blob/f1e1dfe0/exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java
new file mode 100644
index 0000000..e83fc05
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java
@@ -0,0 +1,100 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec;
+
+
+import org.apache.drill.common.exceptions.DrillException;
+import org.apache.drill.test.ConfigBuilder;
+import org.junit.Test;
+import static junit.framework.TestCase.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSSLConfig {
+
+  @Test
+  public void testMissingKeystorePath() throws Exception {
+
+    ConfigBuilder config = new ConfigBuilder();
+    config.put(ExecConstants.HTTP_KEYSTORE_PATH, "");
+    config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "root");
+    try {
+      SSLConfig sslv = new SSLConfig(config.build());
+      fail();
+      //Expected
+    } catch (Exception e) {
+      assertTrue(e instanceof DrillException);
+    }
+  }
+
+  @Test
+  public void testMissingKeystorePassword() throws Exception {
+
+    ConfigBuilder config = new ConfigBuilder();
+    config.put(ExecConstants.HTTP_KEYSTORE_PATH, "/root");
+    config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "");
+    try {
+      SSLConfig sslv = new SSLConfig(config.build());
+      fail();
+      //Expected
+    } catch (Exception e) {
+      assertTrue(e instanceof DrillException);
+    }
+  }
+
+  @Test
+  public void testForKeystoreConfig() throws Exception {
+
+    ConfigBuilder config = new ConfigBuilder();
+    config.put(ExecConstants.HTTP_KEYSTORE_PATH, "/root");
+    config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "root");
+    try {
+      SSLConfig sslv = new SSLConfig(config.build());
+      assertEquals("/root", sslv.getKeyStorePath());
+      assertEquals("root", sslv.getKeyStorePassword());
+    } catch (Exception e) {
+      fail();
+
+    }
+  }
+
+  @Test
+  public void testForBackwardCompatability() throws Exception {
+
+    ConfigBuilder config = new ConfigBuilder();
+    config.put("javax.net.ssl.keyStore", "/root");
+    config.put("javax.net.ssl.keyStorePassword", "root");
+    SSLConfig sslv = new SSLConfig(config.build());
+    assertEquals("/root",sslv.getKeyStorePath());
+    assertEquals("root", sslv.getKeyStorePassword());
+  }
+
+  @Test
+  public void testForTrustStore() throws Exception {
+
+    ConfigBuilder config = new ConfigBuilder();
+    config.put(ExecConstants.HTTP_TRUSTSTORE_PATH, "/root");
+    config.put(ExecConstants.HTTP_TRUSTSTORE_PASSWORD, "root");
+    SSLConfig sslv = new SSLConfig(config.build());
+    assertEquals(true, sslv.hasTrustStorePath());
+    assertEquals(true,sslv.hasTrustStorePassword());
+    assertEquals("/root",sslv.getTrustStorePath());
+    assertEquals("root",sslv.getTrustStorePassword());
+  }
+}
\ No newline at end of file


[03/13] drill git commit: DRILL-5601: Rollup of external sort fixes an improvements

Posted by jn...@apache.org.
DRILL-5601: Rollup of external sort fixes an improvements

- DRILL-5513: Managed External Sort : OOM error during the merge phase
- DRILL-5519: Sort fails to spill and results in an OOM
- DRILL-5522: OOM during the merge and spill process of the managed external sort
- DRILL-5594: Excessive buffer reallocations during merge phase of external sort
- DRILL-5597: Incorrect "bits" vector allocation in nullable vectors allocateNew()
- DRILL-5602: Repeated List Vector fails to initialize the offset vector
- DRILL-5617: Spill file name collisions when spill file is on a shared file system
0 DRILL-5445: bug in repeated map vector deserialization
- Workaround for DRILL-5656: Streaming Agg Batch forces sort to retain in-memory batches past NONE
- Fixes for the "record batch sizer" to handle for UNION, MAP, LIST types

Fixes a longstanding bug in the deserialization of a repeated map
vector read from a spill file. A few minor code cleanups also.

All of the bugs have to do with handling low-memory conditions, and with
correctly estimating the sizes of vectors, even when those vectors come
from the spill file or from an exchange. Hence, the changes for all of
the above issues are interrelated.

Also includes some fixes for tests:

* Certain tests require the ability to enforce the output size of the
memory merge/sort. Restored this option.
* Resolve issue with TestDrillbitResilience
* A particular test injects a fault during in-memory sort, but used a
single-batch input (which does not need the merge phase.)
Rather than introduce a new config property (the earlier solution),
altered the test to use input that returns more than one batch.

Two fixes forBasicPhysicalOpUnitTest — a test that uses JMockit to create a “fake”
fragment context.

1. No drillbit endpoint is available, so the SpillSet change that adds
the node to the spill path failed. Solution was to omit the node path
segment in such tests.

2. The Dynamic UDF registry is null causing a crash. This has nothing
to do with sort. Perhaps some pre-existing error? Anyway, added a check
for this condition.

close #860


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/073ea681
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/073ea681
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/073ea681

Branch: refs/heads/master
Commit: 073ea68197cdc37e2ca9414da96e9df39ec49fcc
Parents: ae123e1
Author: Paul Rogers <pr...@maprtech.com>
Authored: Thu Apr 6 13:57:19 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 .../cache/VectorAccessibleSerializable.java     |   3 +-
 .../drill/exec/coord/ClusterCoordinator.java    |  10 +-
 .../expr/fn/FunctionImplementationRegistry.java |   9 +-
 .../fn/registry/RemoteFunctionRegistry.java     |   8 +
 .../physical/impl/aggregate/InternalBatch.java  |   4 +-
 .../impl/aggregate/StreamingAggBatch.java       |  12 +
 .../impl/aggregate/StreamingAggTemplate.java    |  24 +-
 .../impl/sort/SortRecordBatchBuilder.java       |  13 +-
 .../physical/impl/spill/RecordBatchSizer.java   | 348 ++++++++----
 .../exec/physical/impl/spill/SpillSet.java      |  24 +-
 .../IteratorValidatorBatchIterator.java         |   3 +-
 .../exec/physical/impl/xsort/MSortTemplate.java |   6 +-
 .../physical/impl/xsort/managed/BatchGroup.java |   7 +-
 .../impl/xsort/managed/BufferedBatches.java     |   1 +
 .../impl/xsort/managed/ExternalSortBatch.java   |  74 ++-
 .../impl/xsort/managed/MSortTemplate.java       |   4 +-
 .../physical/impl/xsort/managed/MSorter.java    |   3 +-
 .../impl/xsort/managed/MergeSortWrapper.java    |  18 +-
 .../managed/PriorityQueueCopierTemplate.java    |   1 -
 .../managed/PriorityQueueCopierWrapper.java     |  53 +-
 .../physical/impl/xsort/managed/SortConfig.java |  27 +-
 .../physical/impl/xsort/managed/SortImpl.java   | 271 ++++++----
 .../impl/xsort/managed/SortMemoryManager.java   | 523 ++++++++++++++-----
 .../impl/xsort/managed/SorterWrapper.java       |   3 +-
 .../impl/xsort/managed/SpilledRuns.java         |  31 +-
 .../drill/exec/record/AbstractRecordBatch.java  |   6 +-
 .../drill/exec/record/HyperVectorWrapper.java   |   7 +-
 .../apache/drill/exec/record/RecordBatch.java   |   4 +-
 .../drill/exec/record/VectorInitializer.java    | 154 ++++++
 .../apache/drill/exec/record/WritableBatch.java |   5 +-
 .../org/apache/drill/exec/server/Drillbit.java  |   1 +
 .../src/main/resources/drill-module.conf        |  11 +-
 .../java/org/apache/drill/TestUnionAll.java     |   1 -
 .../exec/cache/TestBatchSerialization.java      |   6 -
 .../physical/impl/window/TestWindowFrame.java   |   5 +
 .../impl/xsort/TestSimpleExternalSort.java      |   7 +-
 .../impl/xsort/TestSortSpillWithException.java  |   8 +-
 .../impl/xsort/managed/SortTestUtilities.java   |   2 +-
 .../physical/impl/xsort/managed/TestCopier.java |   7 +-
 .../managed/TestExternalSortInternals.java      | 224 +++++---
 .../impl/xsort/managed/TestSortImpl.java        |  10 +-
 .../exec/server/TestDrillbitResilience.java     |  38 +-
 .../org/apache/drill/test/ClientFixture.java    | 120 +++++
 .../org/apache/drill/test/ProfileParser.java    |   2 +-
 .../org/apache/drill/test/QueryBuilder.java     |  25 +
 .../src/main/java/io/netty/buffer/DrillBuf.java |  17 +-
 .../drill/exec/memory/AllocationManager.java    |  17 +-
 .../apache/drill/exec/memory/BaseAllocator.java |   1 +
 .../src/main/codegen/includes/vv_imports.ftl    |   2 +
 .../codegen/templates/FixedValueVectors.java    |   4 +-
 .../codegen/templates/NullableValueVectors.java |  13 +-
 .../src/main/codegen/templates/UnionVector.java |  14 +-
 .../templates/VariableLengthVectors.java        |  37 +-
 .../drill/exec/vector/AllocationHelper.java     |  43 +-
 .../drill/exec/vector/BaseDataValueVector.java  |  14 +-
 .../org/apache/drill/exec/vector/BitVector.java |   5 +-
 .../apache/drill/exec/vector/ObjectVector.java  |   9 +-
 .../apache/drill/exec/vector/ValueVector.java   |  11 +-
 .../apache/drill/exec/vector/ZeroVector.java    |   9 +-
 .../exec/vector/complex/AbstractMapVector.java  |  14 +-
 .../vector/complex/BaseRepeatedValueVector.java |  13 +-
 .../drill/exec/vector/complex/ListVector.java   |  13 +-
 .../exec/vector/complex/RepeatedListVector.java |  13 +-
 .../exec/vector/complex/RepeatedMapVector.java  |  25 +-
 64 files changed, 1759 insertions(+), 638 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
index d569ae5..0a78cb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
@@ -35,8 +35,6 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.vector.NullableBigIntVector;
-import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.codahale.metrics.MetricRegistry;
@@ -141,6 +139,7 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable {
   }
 
   // Like above, only preserve the original container and list of value-vectors
+  @SuppressWarnings("resource")
   public void readFromStreamWithContainer(VectorContainer myContainer, InputStream input) throws IOException {
     final VectorContainer container = new VectorContainer();
     final UserBitShared.RecordBatchDef batchDef = UserBitShared.RecordBatchDef.parseDelimitedFrom(input);

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
index ea9593e..e758d6f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
@@ -38,8 +38,12 @@ public abstract class ClusterCoordinator implements AutoCloseable {
       16, 0.75f, 16);
 
   /**
-   * Start the cluster coordinator.  Millis to wait is
-   * @param millisToWait The maximum time to wait before throwing an exception if the cluster coordination service has not successfully started.  Use 0 to wait indefinitely.
+   * Start the cluster coordinator. Millis to wait is
+   *
+   * @param millisToWait
+   *          The maximum time to wait before throwing an exception if the
+   *          cluster coordination service has not successfully started. Use 0
+   *          to wait indefinitely.
    * @throws Exception
    */
   public abstract void start(long millisToWait) throws Exception;
@@ -49,7 +53,7 @@ public abstract class ClusterCoordinator implements AutoCloseable {
   public abstract void unregister(RegistrationHandle handle);
 
   /**
-   * Get a collection of avialable Drillbit endpoints, Thread-safe.
+   * Get a collection of available Drillbit endpoints, Thread-safe.
    * Could be slightly out of date depending on refresh policy.
    *
    * @return A collection of available endpoints.

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
index 8bc6af0..1c399e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
@@ -343,7 +343,9 @@ public class FunctionImplementationRegistry implements FunctionLookupContext, Au
    */
   @SuppressWarnings("resource")
   public boolean syncWithRemoteRegistry(long version) {
-    if (isRegistrySyncNeeded(remoteFunctionRegistry.getRegistryVersion(), localFunctionRegistry.getVersion())) {
+    // Do the version check only if a remote registry exists. It does
+    // not exist for some JMockit-based unit tests.
+    if (isRegistrySyncNeeded()) {
       synchronized (this) {
         long localRegistryVersion = localFunctionRegistry.getVersion();
         if (isRegistrySyncNeeded(remoteFunctionRegistry.getRegistryVersion(), localRegistryVersion))  {
@@ -390,6 +392,11 @@ public class FunctionImplementationRegistry implements FunctionLookupContext, Au
     return version != localFunctionRegistry.getVersion();
   }
 
+  private boolean isRegistrySyncNeeded() {
+    return remoteFunctionRegistry.hasRegistry() &&
+           isRegistrySyncNeeded(remoteFunctionRegistry.getRegistryVersion(), localFunctionRegistry.getVersion());
+  }
+
   /**
    * Checks if local function registry should be synchronized with remote function registry.
    * If remote function registry version is -1, it means that remote function registry is unreachable

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
index 2e5eda2..38d8fcc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
@@ -128,6 +128,14 @@ public class RemoteFunctionRegistry implements AutoCloseable {
     }
   }
 
+  /**
+   * Report whether a remote registry exists. During some unit tests,
+   * no remote registry exists, so the other methods should not be called.
+   * @return true if a remote registry exists, false if this a local-only
+   * instance and no such registry exists
+   */
+  public boolean hasRegistry() { return registry != null; }
+
   public Registry getRegistry(DataChangeVersion version) {
     return registry.get(registry_path, version);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
index 9e96727..1d1d3cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -39,7 +39,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
     this(incoming, null, oContext);
   }
 
-  public InternalBatch(RecordBatch incoming, VectorWrapper[] ignoreWrappers, OperatorContext oContext){
+  public InternalBatch(RecordBatch incoming, VectorWrapper<?>[] ignoreWrappers, OperatorContext oContext){
     switch(incoming.getSchema().getSelectionVectorMode()){
     case FOUR_BYTE:
       this.sv4 = incoming.getSelectionVector4().createNewWrapperCurrent();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index af41438..f7fd1d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -43,6 +43,7 @@ import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator.AggOutcome;
+import org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch;
 import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -87,6 +88,14 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   public StreamingAggBatch(StreamingAggregate popConfig, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
     super(popConfig, context);
     this.incoming = incoming;
+
+    // Sorry. Horrible hack. To release memory after an in-memory sort,
+    // the External Sort normally frees in-memory sorted batches just
+    // before returning NONE. But, this operator needs the batches to
+    // be present, even after NONE. This call puts the ESB into the proper
+    // "mode". A later call explicitly releases the batches.
+
+    ExternalSortBatch.retainSv4OnNone(incoming);
   }
 
   @Override
@@ -111,6 +120,8 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       case STOP:
         state = BatchState.STOP;
         return;
+      default:
+        break;
     }
 
     this.incomingSchema = incoming.getSchema();
@@ -176,6 +187,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
         container.zeroVectors();
       }
       done = true;
+      ExternalSortBatch.releaseBatches(incoming);
       // fall through
     case RETURN_OUTCOME:
       IterOutcome outcome = aggregator.getOutcome();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
index 3417611..fb4d508 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
@@ -36,6 +36,9 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
   private int previousIndex = -1;
   private int underlyingIndex = 0;
   private int currentIndex;
+  /**
+   * Number of records added to the current aggregation group.
+   */
   private long addedRecordCount = 0;
   private IterOutcome outcome;
   private int outputCount = 0;
@@ -163,6 +166,19 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
           previousIndex = currentIndex;
         }
 
+        /**
+         * Hold onto the previous incoming batch. When the incoming uses an
+         * SV4, the InternalBatch DOES NOT clone or transfer the data. Instead,
+         * it clones only the SV4, and assumes that the same hyper-list of
+         * batches will be offered again after the next call to the incoming
+         * next(). This is, in fact, how the SV4 works, so all is fine. The
+         * trick to be aware of, however, is that this MUST BE TRUE even if
+         * the incoming next() returns NONE: the incoming is obligated to continue
+         * to offer the set of batches. That is, the incoming cannot try to be
+         * tidy and release the batches one end-of-data or the following code
+         * will fail, likely with an IndexOutOfBounds exception.
+         */
+
         InternalBatch previous = new InternalBatch(incoming, context);
 
         try {
@@ -178,14 +194,14 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
               lastOutcome = out;
               if (first && addedRecordCount == 0) {
                 return setOkAndReturn();
-              } else if(addedRecordCount > 0) {
+              } else if (addedRecordCount > 0) {
                 outputToBatchPrev(previous, previousIndex, outputCount); // No need to check the return value
-                // (output container full or not) as we are not going to insert anymore records.
+                // (output container full or not) as we are not going to insert any more records.
                 if (EXTRA_DEBUG) {
                   logger.debug("Received no more batches, returning.");
                 }
                 return setOkAndReturn();
-              }else{
+              } else {
                 if (first && out == IterOutcome.OK) {
                   out = IterOutcome.OK_NEW_SCHEMA;
                 }
@@ -330,7 +346,7 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
 
   private void addRecordInc(int index) {
     addRecord(index);
-    this.addedRecordCount++;
+    addedRecordCount++;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
index cd52a91..999fb04 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.AllocationReservation;
+import org.apache.drill.exec.memory.BaseAllocator;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
@@ -244,15 +245,17 @@ public class SortRecordBatchBuilder implements AutoCloseable {
   }
 
   /**
-   * For given record count how much memory does SortRecordBatchBuilder needs for its own purpose. This is used in
-   * ExternalSortBatch to make decisions about whether to spill or not.
+   * For given record count, return the memory that SortRecordBatchBuilder needs
+   * for its own purpose. This is used in ExternalSortBatch to make decisions
+   * about whether to spill or not.
    *
-   * @param recordCount
-   * @return
+   * @param recordCount expected output record count
+   * @return number of bytes needed for an SV4, power-of-two rounded
    */
+
   public static long memoryNeeded(int recordCount) {
     // We need 4 bytes (SV4) for each record. Due to power-of-two allocations, the
     // backing buffer might be twice this size.
-    return recordCount * 2 * 4;
+    return BaseAllocator.nextPowerOfTwo(recordCount * 4);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
index a1b8169..f76757f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
@@ -19,117 +19,221 @@ package org.apache.drill.exec.physical.impl.spill;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
+import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 import org.apache.drill.exec.memory.BaseAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorInitializer;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.RepeatedListVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
 
+import com.google.common.collect.Sets;
+
 /**
  * Given a record batch or vector container, determines the actual memory
  * consumed by each column, the average row, and the entire record batch.
  */
 
 public class RecordBatchSizer {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
 
   /**
    * Column size information.
    */
   public static class ColumnSize {
+    public final String prefix;
     public final MaterializedField metadata;
 
     /**
-     * Assumed size from Drill metadata.
+     * Assumed size from Drill metadata. Note that this information is
+     * 100% bogus. Do not use it.
      */
 
+    @Deprecated
     public int stdSize;
 
     /**
-     * Actual memory consumed by all the vectors associated with this column.
-     */
-
-    public int totalSize;
-
-    /**
      * Actual average column width as determined from actual memory use. This
      * size is larger than the actual data size since this size includes per-
      * column overhead such as any unused vector space, etc.
      */
 
-    public int estSize;
-    public int capacity;
-    public int density;
-    public int dataSize;
-    public boolean variableWidth;
+    public final int estSize;
 
-    public ColumnSize(ValueVector vv) {
-      metadata = vv.getField();
-      stdSize = TypeHelper.getSize(metadata.getType());
+    /**
+     * Number of times the value here (possibly repeated) appears in
+     * the record batch.
+     */
 
-      // Can't get size estimates if this is an empty batch.
-      int rowCount = vv.getAccessor().getValueCount();
-      if (rowCount == 0) {
-        estSize = stdSize;
-        return;
-      }
+    public final int valueCount;
 
-      // Total size taken by all vectors (and underlying buffers)
-      // associated with this vector.
+    /**
+     * The number of elements in the value vector. Consider two cases.
+     * A required or nullable vector has one element per row, so the
+     * <tt>entryCount</tt> is the same as the <tt>valueCount</tt> (which,
+     * in turn, is the same as the row count.) But, if this vector is an
+     * array, then the <tt>valueCount</tt> is the number of columns, while
+     * <tt>entryCount</tt> is the total number of elements in all the arrays
+     * that make up the columns, so <tt>entryCount</tt> will be different than
+     * the <tt>valueCount</tt> (normally larger, but possibly smaller if most
+     * arrays are empty.
+     * <p>
+     * Finally, the column may be part of another list. In this case, the above
+     * logic still applies, but the <tt>valueCount</tt> is the number of entries
+     * in the outer array, not the row count.
+     */
+
+    public int entryCount;
+    public int dataSize;
 
-      totalSize = vv.getAllocatedByteCount();
+    /**
+     * The estimated, average number of elements. For a repeated type,
+     * this is the average entries per array (per repeated element).
+     */
 
-      // Capacity is the number of values that the vector could
-      // contain. This is useful only for fixed-length vectors.
+    public int estElementCount;
+    public final boolean isVariableWidth;
 
-      capacity = vv.getValueCapacity();
+    public ColumnSize(ValueVector v, String prefix, int valueCount) {
+      this.prefix = prefix;
+      this.valueCount = valueCount;
+      metadata = v.getField();
+      isVariableWidth = v instanceof VariableWidthVector;
 
       // The amount of memory consumed by the payload: the actual
       // data stored in the vectors.
 
-      dataSize = vv.getPayloadByteCount();
+      if (v.getField().getDataMode() == DataMode.REPEATED) {
+        buildRepeated(v);
+      }
+      estElementCount = 1;
+      entryCount = 1;
+      switch (metadata.getType().getMinorType()) {
+      case LIST:
+        buildList(v);
+        break;
+      case MAP:
+      case UNION:
+        // No standard size for Union type
+        dataSize = v.getPayloadByteCount(valueCount);
+        break;
+      default:
+        dataSize = v.getPayloadByteCount(valueCount);
+        stdSize = TypeHelper.getSize(metadata.getType());
+      }
+      estSize = roundUp(dataSize, valueCount);
+    }
+
+    private void buildRepeated(ValueVector v) {
 
-      // Determine "density" the number of rows compared to potential
-      // capacity. Low-density batches occur at block boundaries, ends
-      // of files and so on. Low-density batches throw off our estimates
-      // for Varchar columns because we don't know the actual number of
-      // bytes consumed (that information is hidden behind the Varchar
-      // implementation where we can't get at it.)
+      // Repeated vectors are special: they have an associated offset vector
+      // that changes the value count of the contained vectors.
 
-      density = roundUp(dataSize * 100, totalSize);
-      estSize = roundUp(dataSize, rowCount);
-      variableWidth = vv instanceof VariableWidthVector ;
+      @SuppressWarnings("resource")
+      UInt4Vector offsetVector = ((RepeatedValueVector) v).getOffsetVector();
+      buildArray(offsetVector);
+      if (metadata.getType().getMinorType() == MinorType.MAP) {
+
+        // For map, the only data associated with the map vector
+        // itself is the offset vector, if any.
+
+        dataSize = offsetVector.getPayloadByteCount(valueCount);
+      }
+    }
+
+    private void buildList(ValueVector v) {
+      @SuppressWarnings("resource")
+      UInt4Vector offsetVector = ((RepeatedListVector) v).getOffsetVector();
+      buildArray(offsetVector);
+      dataSize = offsetVector.getPayloadByteCount(valueCount);
+    }
+
+    private void buildArray(UInt4Vector offsetVector) {
+      entryCount = offsetVector.getAccessor().get(valueCount);
+      estElementCount = roundUp(entryCount, valueCount);
     }
 
     @Override
     public String toString() {
       StringBuilder buf = new StringBuilder()
+          .append(prefix)
           .append(metadata.getName())
           .append("(type: ")
+          .append(metadata.getType().getMode().name())
+          .append(" ")
           .append(metadata.getType().getMinorType().name())
-          .append(", std col. size: ")
+          .append(", count: ")
+          .append(valueCount);
+      if (metadata.getDataMode() == DataMode.REPEATED) {
+        buf.append(", total entries: ")
+           .append(entryCount)
+           .append(", per-array: ")
+           .append(estElementCount);
+      }
+      buf .append(", std size: ")
           .append(stdSize)
-          .append(", actual col. size: ")
+          .append(", actual size: ")
           .append(estSize)
-          .append(", total size: ")
-          .append(totalSize)
           .append(", data size: ")
           .append(dataSize)
-          .append(", row capacity: ")
-          .append(capacity)
-          .append(", density: ")
-          .append(density)
           .append(")");
       return buf.toString();
     }
+
+    /**
+     * Add a single vector initializer to a collection for the entire batch.
+     * Uses the observed column size information to predict the size needed
+     * when allocating a new vector for the same data.
+     *
+     * @param initializer the vector initializer to hold the hints
+     * for this column
+     */
+
+    private void buildVectorInitializer(VectorInitializer initializer) {
+      int width = 0;
+      switch(metadata.getType().getMinorType()) {
+      case VAR16CHAR:
+      case VARBINARY:
+      case VARCHAR:
+
+        // Subtract out the offset vector width
+        width = estSize - 4;
+
+        // Subtract out the bits (is-set) vector width
+        if (metadata.getDataMode() == DataMode.OPTIONAL) {
+          width -= 1;
+        }
+        break;
+      default:
+        break;
+      }
+      String name = prefix + metadata.getName();
+      if (metadata.getDataMode() == DataMode.REPEATED) {
+        if (width > 0) {
+          // Estimated width is width of entire column. Divide
+          // by element count to get per-element size.
+          initializer.variableWidthArray(name, width / estElementCount, estElementCount);
+        } else {
+          initializer.fixedWidthArray(name, estElementCount);
+        }
+      }
+      else if (width > 0) {
+        initializer.variableWidth(name, width);
+      }
+    }
   }
 
   private List<ColumnSize> columnSizes = new ArrayList<>();
@@ -139,14 +243,16 @@ public class RecordBatchSizer {
    */
   private int rowCount;
   /**
-   * Standard row width using Drill meta-data.
+   * Standard row width using Drill meta-data. Note: this information is
+   * 100% bogus. Do not use it.
    */
+  @Deprecated
   private int stdRowWidth;
   /**
    * Actual batch size summing all buffers used to store data
    * for the batch.
    */
-  private int totalBatchSize;
+  private int accountedMemorySize;
   /**
    * Actual row width computed by dividing total batch memory by the
    * record count.
@@ -162,6 +268,8 @@ public class RecordBatchSizer {
   private int sv2Size;
   private int avgDensity;
 
+  private Set<BufferLedger> ledgers = Sets.newIdentityHashSet();
+
   private int netBatchSize;
 
   public RecordBatchSizer(RecordBatch batch) {
@@ -177,42 +285,58 @@ public class RecordBatchSizer {
   /**
    *  Count the nullable columns; used for memory estimation
    */
-  public int numNullables;
+  public int nullableCount;
+
   /**
+   * Create empirical metadata for a record batch given a vector accessible
+   * (basically, an iterator over the vectors in the batch.)
    *
-   * @param va
+   * @param va iterator over the batch's vectors
    */
+
   public RecordBatchSizer(VectorAccessible va) {
     this(va, null);
   }
 
+  /**
+   * Create empirical metadata for a record batch given a vector accessible
+   * (basically, an iterator over the vectors in the batch) along with a
+   * selection vector for those records. The selection vector is used to
+   * pad the estimated row width with the extra two bytes needed per record.
+   * The selection vector memory is added ot the total memory consumed by
+   * this batch.
+   *
+   * @param va iterator over the batch's vectors
+   * @param sv2 selection vector associated with this batch
+   */
+
   public RecordBatchSizer(VectorAccessible va, SelectionVector2 sv2) {
     rowCount = va.getRecordCount();
     for (VectorWrapper<?> vw : va) {
-      int size = measureColumn(vw.getValueVector());
-      if ( size > maxSize ) { maxSize = size; }
-      if ( vw.getField().isNullable() ) { numNullables++; }
+      measureColumn(vw.getValueVector(), "", rowCount);
+    }
+
+    for (BufferLedger ledger : ledgers) {
+      accountedMemorySize += ledger.getAccountedSize();
     }
 
     if (rowCount > 0) {
-      grossRowWidth = roundUp(totalBatchSize, rowCount);
+      grossRowWidth = roundUp(accountedMemorySize, rowCount);
     }
 
     if (sv2 != null) {
       sv2Size = sv2.getBuffer(false).capacity();
-      grossRowWidth += roundUp(sv2Size, rowCount);
-      netRowWidth += 2;
+      accountedMemorySize += sv2Size;
+      hasSv2 = true;
     }
 
-    int totalDensity = 0;
-    int usableCount = 0;
-    for (ColumnSize colSize : columnSizes) {
-      if ( colSize.density > 0 ) {
-        usableCount++;
-      }
-      totalDensity += colSize.density;
-    }
-    avgDensity = roundUp(totalDensity, usableCount);
+    computeEstimates();
+  }
+
+  private void computeEstimates() {
+    grossRowWidth = roundUp(accountedMemorySize, rowCount);
+    netRowWidth = roundUp(netBatchSize, rowCount);
+    avgDensity = roundUp(netBatchSize * 100, accountedMemorySize);
   }
 
   public void applySv2() {
@@ -220,9 +344,10 @@ public class RecordBatchSizer {
       return;
     }
 
+    hasSv2 = true;
     sv2Size = BaseAllocator.nextPowerOfTwo(2 * rowCount);
-    grossRowWidth += roundUp(sv2Size, rowCount);
-    totalBatchSize += sv2Size;
+    accountedMemorySize += sv2Size;
+    computeEstimates();
   }
 
   /**
@@ -238,36 +363,61 @@ public class RecordBatchSizer {
     if ( arg <= 32 ) { return 32; }
     return 64;
   }
-  private int measureColumn(ValueVector vv) {
-    // Maps consume no size themselves. However, their contained
-    // vectors do consume space, so visit columns recursively.
-    if (vv.getField().getType().getMinorType() == MinorType.MAP) {
-      return expandMap((AbstractMapVector) vv);
-    }
 
-    ColumnSize colSize = new ColumnSize(vv);
-    columnSizes.add(colSize);
+  private void measureColumn(ValueVector v, String prefix, int valueCount) {
 
+    ColumnSize colSize = new ColumnSize(v, prefix, valueCount);
+    columnSizes.add(colSize);
     stdRowWidth += colSize.stdSize;
-    totalBatchSize += colSize.totalSize;
     netBatchSize += colSize.dataSize;
+    maxSize = Math.max(maxSize, colSize.dataSize);
+    if (colSize.metadata.isNullable()) {
+      nullableCount++;
+    }
+
+    // Maps consume no size themselves. However, their contained
+    // vectors do consume space, so visit columns recursively.
+
+    switch (v.getField().getType().getMinorType()) {
+    case MAP:
+      expandMap((AbstractMapVector) v, prefix + v.getField().getName() + ".", colSize.entryCount);
+      break;
+    case LIST:
+      expandList((RepeatedListVector) v, prefix + v.getField().getName() + ".", colSize.entryCount);
+      break;
+    default:
+      v.collectLedgers(ledgers);
+    }
+
     netRowWidth += colSize.estSize;
-    netRowWidthCap50 += ! colSize.variableWidth ? colSize.estSize :
-        8 /* offset vector */ + roundUpToPowerOf2( Math.min(colSize.estSize,50) );
+    netRowWidthCap50 += ! colSize.isVariableWidth ? colSize.estSize :
+        8 /* offset vector */ + roundUpToPowerOf2(Math.min(colSize.estSize,50));
         // above change 8 to 4 after DRILL-5446 is fixed
-    return colSize.estSize;
   }
 
-  private int expandMap(AbstractMapVector mapVector) {
-    int accum = 0;
+  private void expandMap(AbstractMapVector mapVector, String prefix, int valueCount) {
     for (ValueVector vector : mapVector) {
-      accum += measureColumn(vector);
+      measureColumn(vector, prefix, valueCount);
     }
-    return accum;
+
+    // For a repeated map, we need the memory for the offset vector (only).
+    // Map elements are recursively expanded above.
+
+    if (mapVector.getField().getDataMode() == DataMode.REPEATED) {
+      ((RepeatedMapVector) mapVector).getOffsetVector().collectLedgers(ledgers);
+    }
+  }
+
+  private void expandList(RepeatedListVector vector, String prefix, int valueCount) {
+    measureColumn(vector.getDataVector(), prefix, valueCount);
+
+    // Determine memory for the offset vector (only).
+
+    vector.collectLedgers(ledgers);
   }
 
   public static int roundUp(int num, int denom) {
-    if(denom == 0) {
+    if (denom == 0) {
       return 0;
     }
     return (int) Math.ceil((double) num / denom);
@@ -283,8 +433,8 @@ public class RecordBatchSizer {
    * and null marking columns.
    * @return "real" width of the row
    */
-  public int netRowWidthCap50() { return netRowWidthCap50 + numNullables; }
-  public int actualSize() { return totalBatchSize; }
+  public int netRowWidthCap50() { return netRowWidthCap50 + nullableCount; }
+  public int actualSize() { return accountedMemorySize; }
   public boolean hasSv2() { return hasSv2; }
   public int avgDensity() { return avgDensity; }
   public int netSize() { return netBatchSize; }
@@ -304,14 +454,32 @@ public class RecordBatchSizer {
     buf.append( "  Records: " );
     buf.append(rowCount);
     buf.append(", Total size: ");
-    buf.append(totalBatchSize);
-    buf.append(", Gross row width:");
+    buf.append(accountedMemorySize);
+    buf.append(", Data size: ");
+    buf.append(netBatchSize);
+    buf.append(", Gross row width: ");
     buf.append(grossRowWidth);
-    buf.append(", Net row width:");
+    buf.append(", Net row width: ");
     buf.append(netRowWidth);
-    buf.append(", Density:");
+    buf.append(", Density: ");
     buf.append(avgDensity);
     buf.append("}");
     return buf.toString();
   }
+
+  /**
+   * The column size information gathered here represents empirically-derived
+   * schema metadata. Use that metadata to create an instance of a class that
+   * allocates memory for new vectors based on the observed size information.
+   * The caller provides the row count; the size information here provides
+   * column widths, number of elements in each array, etc.
+   */
+
+  public VectorInitializer buildVectorInitializer() {
+    VectorInitializer initializer = new VectorInitializer();
+    for (ColumnSize colSize : columnSizes) {
+      colSize.buildVectorInitializer(initializer);
+    }
+    return initializer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
index 5a02cee..c96c5dc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
@@ -29,6 +29,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.directory.api.util.Strings;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
@@ -36,6 +37,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashAggregate;
 import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.hadoop.conf.Configuration;
@@ -358,10 +360,17 @@ public class SpillSet {
   private long writeBytes;
 
   public SpillSet(FragmentContext context, PhysicalOperator popConfig) {
-    this(context.getConfig(), context.getHandle(), popConfig);
+    this(context.getConfig(), context.getHandle(), popConfig,
+         // Endpoint appears to be null in some tests.
+         context.getDrillbitContext() == null ? null :
+         context.getDrillbitContext().getEndpoint());
   }
 
-  public SpillSet(DrillConfig config, FragmentHandle handle, PhysicalOperator popConfig) {
+  public SpillSet(FragmentContext context, PhysicalOperator popConfig, DrillbitEndpoint ep) {
+    this(context.getConfig(), context.getHandle(), popConfig, ep);
+  }
+
+  public SpillSet(DrillConfig config, FragmentHandle handle, PhysicalOperator popConfig, DrillbitEndpoint ep) {
     String operName;
 
     // Set the spill options from the configuration
@@ -413,7 +422,16 @@ public class SpillSet {
       fileManager = new HadoopFileManager(spillFs);
     }
 
-    spillDirName = String.format("%s_%s_%s-%s_minor%s", QueryIdHelper.getQueryId(handle.getQueryId()),
+    // If provided with a prefix to identify the Drillbit, prepend that to the
+    // spill directory.
+
+    String nodeDir = "";
+    if (ep != null  &&  ep.getAddress() != null) {
+      nodeDir = ep.getAddress() + "-" + ep.getUserPort() + "/";
+    }
+    spillDirName = String.format("%s%s_%s_%s-%s-%s",
+        nodeDir,
+        QueryIdHelper.getQueryId(handle.getQueryId()),
         operName, handle.getMajorFragmentId(), popConfig.getOperatorId(), handle.getMinorFragmentId());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
index 0d7fccc..2be1ed5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
@@ -281,7 +281,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
 
       // Validate schema when available.
       if (batchState == OK || batchState == OK_NEW_SCHEMA) {
-        final BatchSchema prevLastSchema = lastSchema;
         final BatchSchema prevLastNewSchema = lastNewSchema;
 
         lastSchema = incoming.getSchema();
@@ -365,4 +364,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
                       this.getClass().getCanonicalName()));
   }
 
+  public RecordBatch getIncoming() { return incoming; }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
index 34aa46a..2f3d2f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MSortTemplate.java
@@ -26,6 +26,7 @@ import javax.inject.Named;
 
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BaseAllocator;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RecordBatch;
@@ -92,8 +93,9 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
    * @return
    */
   public static long memoryNeeded(final int recordCount) {
-    // We need 4 bytes (SV4) for each record.
-    return recordCount * 4;
+    // We need 4 bytes (SV4) for each record, power of 2 rounded.
+
+    return BaseAllocator.nextPowerOfTwo(recordCount * 4);
   }
 
   private int merge(final int leftStart, final int rightStart, final int rightEnd, final int outStart) {

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
index a74183c..f16bec6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
@@ -212,11 +212,16 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
         reader = VectorSerializer.reader(allocator, inputStream);
       }
       Stopwatch watch = Stopwatch.createStarted();
+      long start = allocator.getAllocatedMemory();
       VectorContainer c =  reader.read();
+      long end = allocator.getAllocatedMemory();
+      logger.trace("Read {} records in {} us; size = {}, memory = {}",
+                   c.getRecordCount(),
+                   watch.elapsed(TimeUnit.MICROSECONDS),
+                   (end - start), end);
       if (schema != null) {
         c = SchemaUtil.coerceContainer(c, schema, allocator);
       }
-      logger.trace("Read {} records in {} us", c.getRecordCount(), watch.elapsed(TimeUnit.MICROSECONDS));
       spilledBatches--;
       currentContainer.zeroVectors();
       Iterator<VectorWrapper<?>> wrapperIterator = c.iterator();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
index 8f1b5e5..5bf8073 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
@@ -106,6 +106,7 @@ public class BufferedBatches {
     sorterWrapper.sortBatch(convertedBatch, sv2);
     bufferBatch(convertedBatch, sv2, batchSize);
   }
+
   /**
    * Convert an incoming batch into the agree-upon format.
    * @param incoming

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
index 1dbddee..ea1d605 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.ops.OperExecContext;
 import org.apache.drill.exec.ops.OperExecContextImpl;
 import org.apache.drill.exec.physical.config.ExternalSort;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
+import org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
 import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
@@ -32,6 +33,7 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaUtil;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ControlsInjectorFactory;
@@ -156,12 +158,18 @@ import org.apache.drill.exec.vector.complex.AbstractContainerVector;
 
 public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
-  protected static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ExternalSortBatch.class);
+
+  // For backward compatibility, masquerade as the original
+  // external sort. Else, some tests don't pass.
+
+  protected static final ControlsInjector injector =
+      ControlsInjectorFactory.getInjector(org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.class);
 
   public static final String INTERRUPTION_AFTER_SORT = "after-sort";
   public static final String INTERRUPTION_AFTER_SETUP = "after-setup";
   public static final String INTERRUPTION_WHILE_SPILLING = "spilling";
   public static final String INTERRUPTION_WHILE_MERGING = "merging";
+  private boolean retainInMemoryBatchesOnNone;
 
   private final RecordBatch incoming;
 
@@ -210,7 +218,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
     SortConfig sortConfig = new SortConfig(context.getConfig());
     SpillSet spillSet = new SpillSet(context.getConfig(), context.getHandle(),
-                                     popConfig);
+                                     popConfig, context.getIdentity());
     OperExecContext opContext = new OperExecContextImpl(context, oContext, popConfig, injector);
     PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(opContext);
     SpilledRuns spilledRuns = new SpilledRuns(opContext, spillSet, copierHolder);
@@ -233,6 +241,11 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     return resultsIterator.getSv4();
   }
 
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return resultsIterator.getSv2();
+  }
+
   /**
    * Called by {@link AbstractRecordBatch} as a fast-path to obtain
    * the first record batch and setup the schema of this batch in order
@@ -307,11 +320,18 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       // Close the iterator here to release any remaining resources such
       // as spill files. This is important when a query has a join: the
       // first branch sort may complete before the second branch starts;
-      // it may be quite a while after returning the last row before the
-      // fragment executor calls this opeator's close method.
-
-      resultsIterator.close();
-      resultsIterator = null;
+      // it may be quite a while after returning the last batch before the
+      // fragment executor calls this operator's close method.
+      //
+      // Note however, that the StreamingAgg operator REQUIRES that the sort
+      // retain the batches behind an SV4 when doing an in-memory sort because
+      // the StreamingAgg retains a reference to that data that it will use
+      // after receiving a NONE result code. See DRILL-5656.
+
+      if (! this.retainInMemoryBatchesOnNone) {
+        resultsIterator.close();
+        resultsIterator = null;
+      }
       return IterOutcome.NONE;
     }
   }
@@ -356,7 +376,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       return IterOutcome.NONE;
     }
 
-    // sort may have prematurely exited due to should continue returning false.
+    // sort may have prematurely exited due to shouldContinue() returning false.
 
     if (! context.shouldContinue()) {
       sortState = SortState.DONE;
@@ -410,7 +430,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
       logger.error("received OUT_OF_MEMORY, trying to spill");
       if (! sortImpl.forceSpill()) {
-        throw UserException.memoryError("Received OUT_OF_MEMORY, but enough batches to spill")
+        throw UserException.memoryError("Received OUT_OF_MEMORY, but not enough batches to spill")
           .build(logger);
       }
       break;
@@ -501,4 +521,40 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       throw ex;
     }
   }
+
+  /**
+   * Workaround for DRILL-5656. We wish to release the batches for an
+   * in-memory sort once data is delivered. Normally, we can release them
+   * just before returning NONE. But, the StreamingAggBatch requires that
+   * the batches still be present on NONE. This method "sniffs" the input
+   * provided, and if the external sort, sets a mode that retains the
+   * batches. Yes, it is a horrible hack. But, necessary until the Drill
+   * iterator protocol can be revised.
+   *
+   * @param incoming the incoming batch for some operator which may
+   * (or may not) be an external sort (or, an external sort wrapped
+   * in a batch iterator validator.)
+   */
+
+  public static void retainSv4OnNone(RecordBatch incoming) {
+    if (incoming instanceof IteratorValidatorBatchIterator) {
+      incoming = ((IteratorValidatorBatchIterator) incoming).getIncoming();
+    }
+    if (incoming instanceof ExternalSortBatch) {
+      ((ExternalSortBatch) incoming).retainInMemoryBatchesOnNone = true;
+    }
+  }
+
+  public static void releaseBatches(RecordBatch incoming) {
+    if (incoming instanceof IteratorValidatorBatchIterator) {
+      incoming = ((IteratorValidatorBatchIterator) incoming).getIncoming();
+    }
+    if (incoming instanceof ExternalSortBatch) {
+      ExternalSortBatch esb = (ExternalSortBatch) incoming;
+      if (esb.resultsIterator != null) {
+        esb.resultsIterator.close();
+        esb.resultsIterator = null;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
index da41e5e..5b07c4a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
@@ -58,7 +58,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
 
   @Override
   public void setup(final FragmentExecContext context, final BufferAllocator allocator, final SelectionVector4 vector4,
-                    final VectorContainer hyperBatch, int outputBatchSize) throws SchemaChangeException{
+                    final VectorContainer hyperBatch, int outputBatchSize, int desiredBatchSize) throws SchemaChangeException{
     // we pass in the local hyperBatch since that is where we'll be reading data.
     Preconditions.checkNotNull(vector4);
     this.vector4 = vector4.createNewWrapperCurrent();
@@ -89,7 +89,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
 
     @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * totalCount);
-    desiredRecordBatchCount = Math.min(outputBatchSize, Character.MAX_VALUE);
+    desiredRecordBatchCount = Math.min(outputBatchSize, desiredBatchSize);
     desiredRecordBatchCount = Math.min(desiredRecordBatchCount, totalCount);
     aux = new SelectionVector4(drillBuf, totalCount, desiredRecordBatchCount);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
index 06bbdea..71ae29e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
@@ -30,7 +30,8 @@ import org.apache.drill.exec.record.selection.SelectionVector4;
  */
 
 public interface MSorter {
-  public void setup(FragmentExecContext context, BufferAllocator allocator, SelectionVector4 vector4, VectorContainer hyperBatch, int outputBatchSize) throws SchemaChangeException;
+  public void setup(FragmentExecContext context, BufferAllocator allocator, SelectionVector4 vector4,
+                    VectorContainer hyperBatch, int outputBatchSize, int desiredBatchSize) throws SchemaChangeException;
   public void sort();
   public SelectionVector4 getSV4();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
index 3ab9af3..9fd995f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
@@ -103,7 +103,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
    * @return the sv4 for this operator
    */
 
-  public void merge(List<BatchGroup.InputBatch> batchGroups) {
+  public void merge(List<BatchGroup.InputBatch> batchGroups, int outputBatchSize) {
 
     // Add the buffered batches to a collection that MSorter can use.
     // The builder takes ownership of the batches and will release them if
@@ -124,7 +124,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
       sv4 = builder.getSv4();
       Sort popConfig = context.getOperatorDefn();
       mSorter = createNewMSorter(popConfig.getOrderings(), MAIN_MAPPING, LEFT_MAPPING, RIGHT_MAPPING);
-      mSorter.setup(context, context.getAllocator(), sv4, destContainer, sv4.getCount());
+      mSorter.setup(context, context.getAllocator(), sv4, destContainer, sv4.getCount(), outputBatchSize);
     } catch (SchemaChangeException e) {
       throw UserException.unsupportedError(e)
             .message("Unexpected schema change - likely code error.")
@@ -139,7 +139,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
     context.injectUnchecked(ExternalSortBatch.INTERRUPTION_AFTER_SORT);
     sv4 = mSorter.getSV4();
 
-    destContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
+//    destContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
   }
 
   private MSorter createNewMSorter(List<Ordering> orderings, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping) {
@@ -147,7 +147,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
     cg.plainJavaCapable(true);
 
     // Uncomment out this line to debug the generated code.
-//  cg.saveCodeForDebugging(true);
+//    cg.saveCodeForDebugging(true);
     ClassGenerator<MSorter> g = cg.getRoot();
     g.setMappingSet(mainMapping);
 
@@ -232,13 +232,9 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
     } catch (RuntimeException e) {
       ex = (ex == null) ? e : ex;
     }
-    try {
-      if (sv4 != null) {
-        sv4.clear();
-      }
-    } catch (RuntimeException e) {
-      ex = (ex == null) ? e : ex;
-    }
+
+    // Sv4 is cleared by the builder, above.
+
     if (ex != null) {
       throw ex;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
index 7a460f5..b48c012 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
@@ -66,7 +66,6 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
 
   @Override
   public int next(int targetRecordCount) {
-    VectorAccessibleUtilities.allocateVectors(outgoing, targetRecordCount);
     for (int outgoingIndex = 0; outgoingIndex < targetRecordCount; outgoingIndex++) {
       if (queueSize == 0) {
         return 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
index 6b71782..6ec8862 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
@@ -34,7 +34,9 @@ import org.apache.drill.exec.ops.OperExecContext;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorInitializer;
 import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
@@ -100,10 +102,12 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
    * @param batchGroupList
    * @param outputContainer
    * @param targetRecordCount
+   * @param allocHelper
    * @return
    */
-  public BatchMerger startMerge(BatchSchema schema, List<? extends BatchGroup> batchGroupList, VectorContainer outputContainer, int targetRecordCount) {
-    return new BatchMerger(this, schema, batchGroupList, outputContainer, targetRecordCount);
+  public BatchMerger startMerge(BatchSchema schema, List<? extends BatchGroup> batchGroupList,
+              VectorContainer outputContainer, int targetRecordCount, VectorInitializer allocHelper) {
+    return new BatchMerger(this, schema, batchGroupList, outputContainer, targetRecordCount, allocHelper);
   }
 
   /**
@@ -195,11 +199,11 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
 
   public static class BatchMerger implements SortResults, AutoCloseable {
 
-    private PriorityQueueCopierWrapper holder;
-    private VectorContainer hyperBatch;
-    private VectorContainer outputContainer;
-    private int targetRecordCount;
-    private int copyCount;
+    private final PriorityQueueCopierWrapper holder;
+    private final VectorContainer hyperBatch;
+    private final VectorContainer outputContainer;
+    private final VectorInitializer allocHelper;
+    private final int targetRecordCount;
     private int batchCount;
     private long estBatchSize;
 
@@ -212,8 +216,8 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
      * @param targetRecordCount number of records for each output batch
      */
     private BatchMerger(PriorityQueueCopierWrapper holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
-                        int targetRecordCount) {
-      this(holder, schema, batchGroupList, new VectorContainer(), targetRecordCount);
+                        int targetRecordCount, VectorInitializer allocHelper) {
+      this(holder, schema, batchGroupList, new VectorContainer(), targetRecordCount, allocHelper);
     }
 
     /**
@@ -224,12 +228,13 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
      * @param batchGroupList the input batches
      * @param outputContainer merges output batch into the given output container
      * @param targetRecordCount number of records for each output batch
+     * @param allocHelper
      */
     private BatchMerger(PriorityQueueCopierWrapper holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
-                        VectorContainer outputContainer, int targetRecordCount) {
+                        VectorContainer outputContainer, int targetRecordCount, VectorInitializer allocHelper) {
       this.holder = holder;
+      this.allocHelper = allocHelper;
       hyperBatch = constructHyperBatch(schema, batchGroupList);
-      copyCount = 0;
       this.targetRecordCount = targetRecordCount;
       this.outputContainer = outputContainer;
       holder.createCopier(hyperBatch, batchGroupList, outputContainer);
@@ -245,29 +250,35 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
 
     @Override
     public boolean next() {
-      Stopwatch w = Stopwatch.createStarted();
       long start = holder.getAllocator().getAllocatedMemory();
+
+      // Allocate an outgoing container the "dumb" way (based on static sizes)
+      // for testing, or the "smart" way (based on actual observed data sizes)
+      // for production code.
+
+      if (allocHelper == null) {
+        VectorAccessibleUtilities.allocateVectors(outputContainer, targetRecordCount);
+      } else {
+        allocHelper.allocateBatch(outputContainer, targetRecordCount);
+      }
+      logger.trace("Initial output batch allocation: {} bytes",
+                   holder.getAllocator().getAllocatedMemory() - start);
+      Stopwatch w = Stopwatch.createStarted();
       int count = holder.copier.next(targetRecordCount);
-      copyCount += count;
       if (count > 0) {
         long t = w.elapsed(TimeUnit.MICROSECONDS);
         batchCount++;
-        logger.trace("Took {} us to merge {} records", t, count);
         long size = holder.getAllocator().getAllocatedMemory() - start;
+        logger.trace("Took {} us to merge {} records, consuming {} bytes of memory",
+                     t, count, size);
         estBatchSize = Math.max(estBatchSize, size);
       } else {
         logger.trace("copier returned 0 records");
       }
 
-      // Identify the schema to be used in the output container. (Since
-      // all merged batches have the same schema, the schema we identify
-      // here should be the same as that which we already had.
+      // Initialize output container metadata.
 
       outputContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-
-      // The copier does not set the record count in the output
-      // container, so do that here.
-
       outputContainer.setRecordCount(count);
 
       return count > 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
index e47d67e..8ae3998 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
@@ -21,6 +21,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 
 public class SortConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
 
   /**
    * Smallest allowed output batch size. The smallest output batch
@@ -62,6 +63,12 @@ public class SortConfig {
 
   private final int bufferedBatchLimit;
 
+  /**
+   * Limit the size of the in-memory merge return batches.
+   * Primarily for testing.
+   */
+
+  private final int mSortBatchSize;
 
   public SortConfig(DrillConfig config) {
 
@@ -101,15 +108,26 @@ public class SortConfig {
     } else {
       bufferedBatchLimit = Math.max(value, 2);
     }
+
+    // Limit on memory merge batch size; primarily for testing
+
+    if (config.hasPath(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE)) {
+      mSortBatchSize = Math.max(1,
+            Math.min(Character.MAX_VALUE,
+                     config.getInt(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE)));
+    } else {
+      mSortBatchSize = Character.MAX_VALUE;
+    }
+
     logConfig();
   }
 
   private void logConfig() {
-    ExternalSortBatch.logger.debug("Config: " +
+    logger.debug("Config: " +
                  "spill file size = {}, spill batch size = {}, " +
-                 "merge limit = {}, merge batch size = {}",
-                  spillFileSize(), spillFileSize(),
-                  mergeLimit(), mergeBatchSize());
+                 "merge batch size = {}, mSort batch size = {}",
+                  spillFileSize, spillBatchSize,
+                  mergeBatchSize, mSortBatchSize);
   }
 
   public long maxMemory() { return maxMemory; }
@@ -118,4 +136,5 @@ public class SortConfig {
   public int spillBatchSize() { return spillBatchSize; }
   public int mergeBatchSize() { return mergeBatchSize; }
   public int getBufferedBatchLimit() { return bufferedBatchLimit; }
+  public int getMSortBatchSize() { return mSortBatchSize; }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
index 6f0da3d..7420402 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
@@ -27,9 +27,12 @@ import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
 import org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeTask;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorInitializer;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 
@@ -70,6 +73,103 @@ public class SortImpl {
     SelectionVector4 getSv4();
   }
 
+  public static class EmptyResults implements SortResults {
+
+    private final VectorContainer dest;
+
+    public EmptyResults(VectorContainer dest) {
+      dest.setRecordCount(0);
+      dest.buildSchema(SelectionVectorMode.NONE);
+      this.dest = dest;
+    }
+
+    @Override
+    public boolean next() { return false; }
+
+    @Override
+    public void close() { }
+
+    @Override
+    public int getBatchCount() { return 0; }
+
+    @Override
+    public int getRecordCount() { return 0; }
+
+    @Override
+    public SelectionVector4 getSv4() { return null; }
+
+    @Override
+    public SelectionVector2 getSv2() { return null; }
+
+    @Override
+    public VectorContainer getContainer() { return dest; }
+  }
+
+
+  /**
+   * Return results for a single input batch. No merge is needed;
+   * the original (sorted) input batch is simply passed as the result.
+   * Note that this version requires replacing the operator output
+   * container with the batch container. (Vector ownership transfer
+   * was already done when accepting the input batch.)
+   */
+
+  public static class SingleBatchResults implements SortResults {
+
+    private boolean done;
+    private final VectorContainer outputContainer;
+    private final BatchGroup.InputBatch batch;
+
+    public SingleBatchResults(BatchGroup.InputBatch batch, VectorContainer outputContainer) {
+      this.batch = batch;
+      this.outputContainer = outputContainer;
+    }
+
+    @Override
+    public boolean next() {
+      if (done) {
+        return false;
+      }
+
+      // The following implementation is wrong. Must transfer buffers,
+      // not vectors. The output container already contains vectors
+      // for the output schema.
+
+      for (VectorWrapper<?> vw : batch.getContainer()) {
+        outputContainer.add(vw.getValueVector());
+      }
+      outputContainer.buildSchema(SelectionVectorMode.TWO_BYTE);
+      outputContainer.setRecordCount(batch.getRecordCount());
+      done = true;
+      return true;
+    }
+
+    @Override
+    public void close() {
+      try {
+        batch.close();
+      } catch (IOException e) {
+        // Should never occur for an input batch
+        throw new IllegalStateException(e);
+      }
+    }
+
+    @Override
+    public int getBatchCount() { return 1; }
+
+    @Override
+    public int getRecordCount() { return outputContainer.getRecordCount(); }
+
+    @Override
+    public SelectionVector4 getSv4() { return null; }
+
+    @Override
+    public SelectionVector2 getSv2() { return batch.getSv2(); }
+
+    @Override
+    public VectorContainer getContainer() { return outputContainer; }
+  }
+
   private final SortConfig config;
   private final SortMetrics metrics;
   private final SortMemoryManager memManager;
@@ -88,7 +188,12 @@ public class SortImpl {
 
   private final BufferedBatches bufferedBatches;
 
-  public SortImpl(OperExecContext opContext, SortConfig sortConfig, SpilledRuns spilledRuns, VectorContainer batch) {
+  private RecordBatchSizer sizer;
+
+  private VectorInitializer allocHelper;
+
+  public SortImpl(OperExecContext opContext, SortConfig sortConfig,
+                  SpilledRuns spilledRuns, VectorContainer batch) {
     this.context = opContext;
     outputBatch = batch;
     this.spilledRuns = spilledRuns;
@@ -103,7 +208,10 @@ public class SortImpl {
     // limit will reduce the probability that random chance causes the allocator
     // to kill the query because of a small, spurious over-allocation.
 
-    allocator.setLimit((long)(allocator.getLimit() * 1.10));
+    long maxMem = memManager.getMemoryLimit();
+    long newMax = (long)(maxMem * 1.10);
+    allocator.setLimit(newMax);
+    logger.debug("Config: Resetting allocator to 10% safety margin: {}", newMax);
   }
 
   public void setSchema(BatchSchema schema) {
@@ -138,16 +246,16 @@ public class SortImpl {
     // ownership. Allows us to figure out if we need to spill first,
     // to avoid overflowing memory simply due to ownership transfer.
 
-    RecordBatchSizer sizer = analyzeIncomingBatch(incoming);
+   analyzeIncomingBatch(incoming);
 
     // The heart of the external sort operator: spill to disk when
     // the in-memory generation exceeds the allowed memory limit.
     // Preemptively spill BEFORE accepting the new batch into our memory
-    // pool. The allocator will throw an OOM exception if we accept the
-    // batch when we are near the limit - despite the fact that the batch
-    // is already in memory and no new memory is allocated during the transfer.
+    // pool. Although the allocator will allow us to exceed the memory limit
+    // during the transfer, we immediately follow the transfer with an SV2
+    // allocation that will fail if we are over the allocation limit.
 
-    if ( isSpillNeeded(sizer.actualSize())) {
+    if (isSpillNeeded(sizer.actualSize())) {
       spillFromMemory();
     }
 
@@ -172,7 +280,12 @@ public class SortImpl {
     // (which may exclude some records due to filtering.)
 
     validateBatchSize(sizer.actualSize(), batchSize);
-    memManager.updateEstimates((int) batchSize, sizer.netRowWidth(), sizer.rowCount());
+    if (memManager.updateEstimates((int) batchSize, sizer.netRowWidth(), sizer.rowCount())) {
+
+      // If estimates changed, discard the helper based on the old estimates.
+
+      allocHelper = null;
+    }
   }
 
   /**
@@ -181,13 +294,12 @@ public class SortImpl {
    * @return an analysis of the incoming batch
    */
 
-  private RecordBatchSizer analyzeIncomingBatch(VectorAccessible incoming) {
-    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
+  private void analyzeIncomingBatch(VectorAccessible incoming) {
+    sizer = new RecordBatchSizer(incoming);
     sizer.applySv2();
     if (metrics.getInputBatchCount() == 0) {
       logger.debug("{}", sizer.toString());
     }
-    return sizer;
   }
 
   /**
@@ -195,20 +307,32 @@ public class SortImpl {
    * Spilling is driven purely by memory availability (and an optional
    * batch limit for testing.)
    *
-   * @return true if spilling is needed, false otherwise
+   * @return true if spilling is needed (and possible), false otherwise
    */
 
   private boolean isSpillNeeded(int incomingSize) {
 
+    if (bufferedBatches.size() >= config.getBufferedBatchLimit()) {
+      return true;
+    }
+
     // Can't spill if less than two batches else the merge
     // can't make progress.
 
+    final boolean spillNeeded = memManager.isSpillNeeded(allocator.getAllocatedMemory(), incomingSize);
     if (bufferedBatches.size() < 2) {
-      return false; }
 
-    if (bufferedBatches.size() >= config.getBufferedBatchLimit()) {
-      return true; }
-    return memManager.isSpillNeeded(allocator.getAllocatedMemory(), incomingSize);
+      // If we can't fit the batch into memory, then place a definite error
+      // message into the log to simplify debugging.
+
+      if (spillNeeded) {
+        logger.error("Insufficient memory to merge two batches. Incoming batch size: {}, available memory: {}",
+                     incomingSize, memManager.freeMemory(allocator.getAllocatedMemory()));
+      }
+      return false;
+    }
+
+    return spillNeeded;
   }
 
   private void validateBatchSize(long actualBatchSize, long memoryDelta) {
@@ -235,46 +359,23 @@ public class SortImpl {
 
     // Do the actual spill.
 
-    logger.trace("Spilling {} of {} batches, memory = {}",
+    logger.trace("Spilling {} of {} batches, allocated memory = {} bytes",
         batchesToSpill.size(), startCount,
         allocator.getAllocatedMemory());
     int spillBatchRowCount = memManager.getSpillBatchRowCount();
-    spilledRuns.mergeAndSpill(batchesToSpill, spillBatchRowCount);
+    spilledRuns.mergeAndSpill(batchesToSpill, spillBatchRowCount, allocHelper());
     metrics.incrSpillCount();
   }
 
-  public SortMetrics getMetrics() { return metrics; }
-
-  public static class EmptyResults implements SortResults {
-
-    private final VectorContainer dest;
-
-    public EmptyResults(VectorContainer dest) {
-      this.dest = dest;
+  private VectorInitializer allocHelper() {
+    if (allocHelper == null) {
+      allocHelper = sizer.buildVectorInitializer();
     }
-
-    @Override
-    public boolean next() { return false; }
-
-    @Override
-    public void close() { }
-
-    @Override
-    public int getBatchCount() { return 0; }
-
-    @Override
-    public int getRecordCount() { return 0; }
-
-    @Override
-    public SelectionVector4 getSv4() { return null; }
-
-    @Override
-    public SelectionVector2 getSv2() { return null; }
-
-    @Override
-    public VectorContainer getContainer() { return dest; }
+    return allocHelper;
   }
 
+  public SortMetrics getMetrics() { return metrics; }
+
   public SortResults startMerge() {
     if (metrics.getInputRowCount() == 0) {
       return new EmptyResults(outputBatch);
@@ -284,72 +385,22 @@ public class SortImpl {
         metrics.getInputBatchCount(), spilledRuns.size(),
         metrics.getInputBytes());
 
-    // Do the merge of the loaded batches. The merge can be done entirely in memory if
-    // the results fit; else we have to do a disk-based merge of
-    // pre-sorted spilled batches.
+    // Do the merge of the loaded batches. The merge can be done entirely in
+    // memory if the results fit; else we have to do a disk-based merge of
+    // pre-sorted spilled batches. Special case the single-batch query;
+    // this accelerates small, quick queries.
+    //
+    // Note: disabling this optimization because it turns out to be
+    // quite hard to transfer a set of vectors from one place to another.
 
-    boolean optimizeOn = true; // Debug only
-    if (optimizeOn && metrics.getInputBatchCount() == 1) {
+    /* if (metrics.getInputBatchCount() == 1) {
       return singleBatchResult();
-    } else if (canUseMemoryMerge()) {
+    } else */ if (canUseMemoryMerge()) {
       return mergeInMemory();
     } else {
       return mergeSpilledRuns();
     }
   }
-
-  /**
-   * Return results for a single input batch. No merge is needed;
-   * the original (sorted) input batch is simply passed as the result.
-   * Note that this version requires replacing the operator output
-   * container with the batch container. (Vector ownership transfer
-   * was already done when accepting the input batch.)
-   */
-
-  public static class SingleBatchResults implements SortResults {
-
-    private boolean done;
-    private final BatchGroup.InputBatch batch;
-
-    public SingleBatchResults(BatchGroup.InputBatch batch) {
-      this.batch = batch;
-    }
-
-    @Override
-    public boolean next() {
-      if (done) {
-        return false;
-      }
-      done = true;
-      return true;
-    }
-
-    @Override
-    public void close() {
-      try {
-        batch.close();
-      } catch (IOException e) {
-        // Should never occur for an input batch
-        throw new IllegalStateException(e);
-      }
-    }
-
-    @Override
-    public int getBatchCount() { return 1; }
-
-    @Override
-    public int getRecordCount() { return batch.getRecordCount(); }
-
-    @Override
-    public SelectionVector4 getSv4() { return null; }
-
-    @Override
-    public SelectionVector2 getSv2() { return batch.getSv2(); }
-
-    @Override
-    public VectorContainer getContainer() {return batch.getContainer(); }
-  }
-
   /**
    * Input consists of a single batch. Just return that batch as
    * the output.
@@ -358,7 +409,7 @@ public class SortImpl {
 
   private SortResults singleBatchResult() {
     List<InputBatch> batches = bufferedBatches.removeAll();
-    return new SingleBatchResults(batches.get(0));
+    return new SingleBatchResults(batches.get(0), outputBatch);
   }
 
   /**
@@ -413,7 +464,7 @@ public class SortImpl {
 
     MergeSortWrapper memoryMerge = new MergeSortWrapper(context, outputBatch);
     try {
-      memoryMerge.merge(bufferedBatches.removeAll());
+      memoryMerge.merge(bufferedBatches.removeAll(), config.getMSortBatchSize());
     } catch (Throwable t) {
       memoryMerge.close();
       throw t;
@@ -461,13 +512,13 @@ public class SortImpl {
     }
 
     int mergeRowCount = memManager.getMergeBatchRowCount();
-    return spilledRuns.finalMerge(bufferedBatches.removeAll(), outputBatch, mergeRowCount);
+    return spilledRuns.finalMerge(bufferedBatches.removeAll(), outputBatch, mergeRowCount, allocHelper);
   }
 
   private void mergeRuns(int targetCount) {
     long mergeMemoryPool = memManager.getMergeMemoryLimit();
     int spillBatchRowCount = memManager.getSpillBatchRowCount();
-    spilledRuns.mergeRuns(targetCount, mergeMemoryPool, spillBatchRowCount);
+    spilledRuns.mergeRuns(targetCount, mergeMemoryPool, spillBatchRowCount, allocHelper);
     metrics.incrMergeCount();
   }
 


[07/13] drill git commit: DRILL-4735: ConvertCountToDirectScan rule enhancements

Posted by jn...@apache.org.
DRILL-4735: ConvertCountToDirectScan rule enhancements

1. ConvertCountToDirectScan rule will be applicable for 2 or more COUNT aggregates.
To achieve this DynamicPojoRecordReader was added which accepts any number of columns,
on the contrary with PojoRecordReader which depends on class fields.
AbstractPojoRecordReader class was added to factor out common logic for these two readers.

2. ConvertCountToDirectScan will distinguish between missing, directory and implicit columns.
For missing columns count will be set 0, for implicit to the total records count
since implicit columns are based on files and there is no data without a file.
If directory column will be encountered, rule won't be applied.
CountsCollector class was introduced to encapsulate counts collection logic.

3. MetadataDirectGroupScan class was introduced to indicate to the user when metadata was used
during calculation and for which files it was applied.

DRILL-4735: Changes after code review.

close #900


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/8b564235
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/8b564235
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/8b564235

Branch: refs/heads/master
Commit: 8b5642353505d1001d7ec3590a07ad1144ecf7f3
Parents: 5c57b50
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Jul 20 19:26:44 2017 +0300
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 .../impl/project/ProjectRecordBatch.java        |   4 +-
 .../physical/ConvertCountToDirectScan.java      | 242 ++++++++++-----
 .../drill/exec/planner/sql/DirectPlan.java      |  16 +-
 .../planner/sql/handlers/ShowFileHandler.java   |   2 +-
 .../apache/drill/exec/store/ColumnExplorer.java | 261 ++++++++++++++++
 .../exec/store/ImplicitColumnExplorer.java      | 219 --------------
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |   7 +-
 .../exec/store/direct/DirectGroupScan.java      |   7 +-
 .../store/direct/MetadataDirectGroupScan.java   |  86 ++++++
 .../ischema/InfoSchemaRecordGenerator.java      |  12 +-
 .../exec/store/parquet/ParquetGroupScan.java    |   4 +-
 .../store/parquet/ParquetScanBatchCreator.java  |   6 +-
 .../store/pojo/AbstractPojoRecordReader.java    | 157 ++++++++++
 .../exec/store/pojo/AbstractPojoWriter.java     |  64 ++++
 .../drill/exec/store/pojo/AbstractWriter.java   |  62 ----
 .../store/pojo/DynamicPojoRecordReader.java     |  71 +++++
 .../drill/exec/store/pojo/PojoRecordReader.java | 187 +++---------
 .../drill/exec/store/pojo/PojoWriter.java       |  38 ++-
 .../drill/exec/store/pojo/PojoWriters.java      | 296 +++++++++++++++++++
 .../apache/drill/exec/store/pojo/Writers.java   | 274 -----------------
 .../exec/store/sys/SystemTableBatchCreator.java |   6 +-
 .../drill/TestFunctionsWithTypeExpoQueries.java |   6 +-
 .../logical/TestConvertCountToDirectScan.java   |  82 ++++-
 23 files changed, 1287 insertions(+), 822 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 676849a..6baf070 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -58,7 +58,7 @@ import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.store.ImplicitColumnExplorer;
+import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.ValueVector;
@@ -500,7 +500,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
   }
 
   private boolean isImplicitFileColumn(ValueVector vvIn) {
-    return ImplicitColumnExplorer.initImplicitFileColumns(context.getOptions()).get(vvIn.getField().getName()) != null;
+    return ColumnExplorer.initImplicitFileColumns(context.getOptions()).get(vvIn.getField().getName()) != null;
   }
 
   private List<NamedExpression> getExpressionList() {

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
index 879d0f7..961816e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,15 +18,21 @@
 
 package org.apache.drill.exec.planner.physical;
 
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelRecordType;
@@ -35,37 +41,41 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
-import org.apache.drill.exec.store.direct.DirectGroupScan;
-import org.apache.drill.exec.store.pojo.PojoRecordReader;
+import org.apache.drill.exec.store.ColumnExplorer;
 
-import com.google.common.collect.Lists;
+import org.apache.drill.exec.store.direct.MetadataDirectGroupScan;
+import org.apache.drill.exec.store.pojo.DynamicPojoRecordReader;
 
 /**
- * This rule will convert
- *   " select count(*)  as mycount from table "
- * or " select count( not-nullable-expr) as mycount from table "
- *   into
- *
+ * <p>
+ * This rule will convert <b>" select count(*)  as mycount from table "</b>
+ * or <b>" select count(not-nullable-expr) as mycount from table "</b> into
+ * <pre>
  *    Project(mycount)
  *         \
  *    DirectGroupScan ( PojoRecordReader ( rowCount ))
- *
- * or
- *    " select count(column) as mycount from table "
- *    into
+ *</pre>
+ * or <b>" select count(column) as mycount from table "</b> into
+ * <pre>
  *      Project(mycount)
  *           \
  *            DirectGroupScan (PojoRecordReader (columnValueCount))
+ *</pre>
+ * Rule can be applied if query contains multiple count expressions.
+ * <b>" select count(column1), count(column2), count(*) from table "</b>
+ * </p>
  *
+ * <p>
  * Currently, only parquet group scan has the exact row count and column value count,
  * obtained from parquet row group info. This will save the cost to
  * scan the whole parquet files.
+ * </p>
  */
-
 public class ConvertCountToDirectScan extends Prule {
 
   public static final RelOptRule AGG_ON_PROJ_ON_SCAN = new ConvertCountToDirectScan(
@@ -77,6 +87,8 @@ public class ConvertCountToDirectScan extends Prule {
       RelOptHelper.some(DrillAggregateRel.class,
                             RelOptHelper.any(DrillScanRel.class)), "Agg_on_scan");
 
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConvertCountToDirectScan.class);
+
   /** Creates a SplunkPushDownRule. */
   protected ConvertCountToDirectScan(RelOptRuleOperand rule, String id) {
     super(rule, "ConvertCountToDirectScan:" + id);
@@ -85,40 +97,85 @@ public class ConvertCountToDirectScan extends Prule {
   @Override
   public void onMatch(RelOptRuleCall call) {
     final DrillAggregateRel agg = (DrillAggregateRel) call.rel(0);
-    final DrillScanRel scan = (DrillScanRel) call.rel(call.rels.length -1);
-    final DrillProjectRel proj = call.rels.length == 3 ? (DrillProjectRel) call.rel(1) : null;
+    final DrillScanRel scan = (DrillScanRel) call.rel(call.rels.length - 1);
+    final DrillProjectRel project = call.rels.length == 3 ? (DrillProjectRel) call.rel(1) : null;
 
     final GroupScan oldGrpScan = scan.getGroupScan();
     final PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
 
-    // Only apply the rule when :
+    // Only apply the rule when:
     //    1) scan knows the exact row count in getSize() call,
     //    2) No GroupBY key,
-    //    3) only one agg function (Check if it's count(*) below).
-    //    4) No distinct agg call.
+    //    3) No distinct agg call.
     if (!(oldGrpScan.getScanStats(settings).getGroupScanProperty().hasExactRowCount()
         && agg.getGroupCount() == 0
-        && agg.getAggCallList().size() == 1
         && !agg.containsDistinctCall())) {
       return;
     }
 
-    AggregateCall aggCall = agg.getAggCallList().get(0);
+    Map<String, Long> result = collectCounts(settings, agg, scan, project);
+    logger.trace("Calculated the following aggregate counts: ", result);
+    // if could not determine the counts, rule won't be applied
+    if (result.isEmpty()) {
+      return;
+    }
+
+    final RelDataType scanRowType = constructDataType(agg, result.keySet());
+
+    final DynamicPojoRecordReader<Long> reader = new DynamicPojoRecordReader<>(
+        buildSchema(scanRowType.getFieldNames()),
+        Collections.singletonList((List<Long>) new ArrayList<>(result.values())));
 
-    if (aggCall.getAggregation().getName().equals("COUNT") ) {
+    final ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, 1, 1, scanRowType.getFieldCount());
+    final GroupScan directScan = new MetadataDirectGroupScan(reader, oldGrpScan.getFiles(), scanStats);
+
+    final ScanPrel newScan = ScanPrel.create(scan,
+        scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), directScan,
+        scanRowType);
+
+    final ProjectPrel newProject = new ProjectPrel(agg.getCluster(), agg.getTraitSet().plus(Prel.DRILL_PHYSICAL)
+        .plus(DrillDistributionTrait.SINGLETON), newScan, prepareFieldExpressions(scanRowType), agg.getRowType());
+
+    call.transformTo(newProject);
+  }
+
+  /**
+   * Collects counts for each aggregation call.
+   * Will return empty result map if was not able to determine count for at least one aggregation call,
+   *
+   * For each aggregate call will determine if count can be calculated. Collects counts only for COUNT function.
+   * For star, not null expressions and implicit columns sets count to total record number.
+   * For other cases obtains counts from group scan operator. Also count can not be calculated for parition columns.
+   *
+   * @param agg aggregate relational expression
+   * @param scan scan relational expression
+   * @param project project relational expression
+   * @return result map where key is count column name, value is count value
+   */
+  private Map<String, Long> collectCounts(PlannerSettings settings, DrillAggregateRel agg, DrillScanRel scan, DrillProjectRel project) {
+    final Set<String> implicitColumnsNames = ColumnExplorer.initImplicitFileColumns(settings.getOptions()).keySet();
+    final GroupScan oldGrpScan = scan.getGroupScan();
+    final long totalRecordCount = oldGrpScan.getScanStats(settings).getRecordCount();
+    final LinkedHashMap<String, Long> result = new LinkedHashMap<>();
+
+    for (int i = 0; i < agg.getAggCallList().size(); i++) {
+      AggregateCall aggCall = agg.getAggCallList().get(i);
+    //for (AggregateCall aggCall : agg.getAggCallList()) {
+      long cnt;
+
+      // rule can be applied only for count function, return empty counts
+      if (!"count".equalsIgnoreCase(aggCall.getAggregation().getName()) ) {
+        return ImmutableMap.of();
+      }
+
+      if (containsStarOrNotNullInput(aggCall, agg)) {
+        cnt = totalRecordCount;
 
-      long cnt = 0;
-      //  count(*)  == >  empty arg  ==>  rowCount
-      //  count(Not-null-input) ==> rowCount
-      if (aggCall.getArgList().isEmpty() ||
-          (aggCall.getArgList().size() == 1 &&
-           ! agg.getInput().getRowType().getFieldList().get(aggCall.getArgList().get(0).intValue()).getType().isNullable())) {
-        cnt = (long) oldGrpScan.getScanStats(settings).getRecordCount();
       } else if (aggCall.getArgList().size() == 1) {
-      // count(columnName) ==> Agg ( Scan )) ==> columnValueCount
+        // count(columnName) ==> Agg ( Scan )) ==> columnValueCount
         int index = aggCall.getArgList().get(0);
 
-        if (proj != null) {
+        if (project != null) {
           // project in the middle of Agg and Scan : Only when input of AggCall is a RexInputRef in Project, we find the index of Scan's field.
           // For instance,
           // Agg - count($0)
@@ -127,67 +184,108 @@ public class ConvertCountToDirectScan extends Prule {
           //    \
           //   Scan (col1, col2).
           // return count of "col2" in Scan's metadata, if found.
-
-          if (proj.getProjects().get(index) instanceof RexInputRef) {
-            index = ((RexInputRef) proj.getProjects().get(index)).getIndex();
-          } else {
-            return;  // do not apply for all other cases.
+          if (!(project.getProjects().get(index) instanceof RexInputRef)) {
+            return ImmutableMap.of(); // do not apply for all other cases.
           }
+
+          index = ((RexInputRef) project.getProjects().get(index)).getIndex();
         }
 
         String columnName = scan.getRowType().getFieldNames().get(index).toLowerCase();
 
-        cnt = oldGrpScan.getColumnValueCount(SchemaPath.getSimplePath(columnName));
-        if (cnt == GroupScan.NO_COLUMN_STATS) {
-          // if column stats are not available don't apply this rule
-          return;
+        // for implicit column count will the same as total record count
+        if (implicitColumnsNames.contains(columnName)) {
+          cnt = totalRecordCount;
+        } else {
+          SchemaPath simplePath = SchemaPath.getSimplePath(columnName);
+
+          if (ColumnExplorer.isPartitionColumn(settings.getOptions(), simplePath)) {
+            return ImmutableMap.of();
+          }
+
+          cnt = oldGrpScan.getColumnValueCount(simplePath);
+          if (cnt == GroupScan.NO_COLUMN_STATS) {
+            // if column stats is not available don't apply this rule, return empty counts
+            return ImmutableMap.of();
+          }
         }
       } else {
-        return; // do nothing.
+        return ImmutableMap.of();
       }
 
-      RelDataType scanRowType = getCountDirectScanRowType(agg.getCluster().getTypeFactory());
-
-      final ScanPrel newScan = ScanPrel.create(scan,
-          scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), getCountDirectScan(cnt),
-          scanRowType);
-
-      List<RexNode> exprs = Lists.newArrayList();
-      exprs.add(RexInputRef.of(0, scanRowType));
-
-      final ProjectPrel newProj = new ProjectPrel(agg.getCluster(), agg.getTraitSet().plus(Prel.DRILL_PHYSICAL)
-          .plus(DrillDistributionTrait.SINGLETON), newScan, exprs, agg.getRowType());
-
-      call.transformTo(newProj);
+      String name = "count" + i + "$" + (aggCall.getName() == null ? aggCall.toString() : aggCall.getName());
+      result.put(name, cnt);
     }
 
+    return ImmutableMap.copyOf(result);
   }
 
   /**
-   * Class to represent the count aggregate result.
+   * Checks if aggregate call contains star or non-null expression:
+   * <pre>
+   * count(*)  == >  empty arg  ==>  rowCount
+   * count(Not-null-input) ==> rowCount
+   * </pre>
+   *
+   * @param aggregateCall aggregate call
+   * @param aggregate aggregate relation expression
+   * @return true of aggregate call contains star or non-null expression
    */
-  public static class CountQueryResult {
-    public long count;
-
-    public CountQueryResult(long cnt) {
-      this.count = cnt;
-    }
+  private boolean containsStarOrNotNullInput(AggregateCall aggregateCall, DrillAggregateRel aggregate) {
+    return aggregateCall.getArgList().isEmpty() ||
+        (aggregateCall.getArgList().size() == 1 &&
+            !aggregate.getInput().getRowType().getFieldList().get(aggregateCall.getArgList().get(0)).getType().isNullable());
   }
 
-  private RelDataType getCountDirectScanRowType(RelDataTypeFactory typeFactory) {
-    List<RelDataTypeField> fields = Lists.newArrayList();
-    fields.add(new RelDataTypeFieldImpl("count", 0, typeFactory.createSqlType(SqlTypeName.BIGINT)));
-
+  /**
+   * For each aggregate call creates field based on its name with bigint type.
+   * Constructs record type for created fields.
+   *
+   * @param aggregateRel aggregate relation expression
+   * @param fieldNames field names
+   * @return record type
+   */
+  private RelDataType constructDataType(DrillAggregateRel aggregateRel, Collection<String> fieldNames) {
+    List<RelDataTypeField> fields = new ArrayList<>();
+    Iterator<String> filedNamesIterator = fieldNames.iterator();
+    int fieldIndex = 0;
+    while (filedNamesIterator.hasNext()) {
+      RelDataTypeField field = new RelDataTypeFieldImpl(
+          filedNamesIterator.next(),
+          fieldIndex++,
+          aggregateRel.getCluster().getTypeFactory().createSqlType(SqlTypeName.BIGINT));
+      fields.add(field);
+    }
     return new RelRecordType(fields);
   }
 
-  private GroupScan getCountDirectScan(long cnt) {
-    CountQueryResult res = new CountQueryResult(cnt);
-
-    PojoRecordReader<CountQueryResult> reader = new PojoRecordReader<CountQueryResult>(CountQueryResult.class,
-        Collections.singleton(res).iterator());
+  /**
+   * Builds schema based on given field names.
+   * Type for each schema is set to long.class.
+   *
+   * @param fieldNames field names
+   * @return schema
+   */
+  private LinkedHashMap<String, Class<?>> buildSchema(List<String> fieldNames) {
+    LinkedHashMap<String, Class<?>> schema = new LinkedHashMap<>();
+    for (String fieldName: fieldNames) {
+      schema.put(fieldName, long.class);
+    }
+    return schema;
+  }
 
-    return new DirectGroupScan(reader);
+  /**
+   * For each field creates row expression.
+   *
+   * @param rowType row type
+   * @return list of row expressions
+   */
+  private List<RexNode> prepareFieldExpressions(RelDataType rowType) {
+    List<RexNode> expressions = new ArrayList<>();
+    for (int i = 0; i < rowType.getFieldCount(); i++) {
+      expressions.add(RexInputRef.of(i, rowType));
+    }
+    return expressions;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
index d40e0d7..3e1d6c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.planner.sql;
 
-import java.util.Collections;
-import java.util.Iterator;
-
 import org.apache.drill.common.logical.PlanProperties;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
 import org.apache.drill.common.logical.PlanProperties.PlanPropertiesBuilder;
@@ -33,6 +30,9 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.direct.DirectGroupScan;
 import org.apache.drill.exec.store.pojo.PojoRecordReader;
 
+import java.util.Collections;
+import java.util.List;
+
 public class DirectPlan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectPlan.class);
 
@@ -43,12 +43,12 @@ public class DirectPlan {
 
   @SuppressWarnings("unchecked")
   public static <T> PhysicalPlan createDirectPlan(QueryContext context, T obj){
-    Iterator<T> iter = (Iterator<T>) Collections.singleton(obj).iterator();
-    return createDirectPlan(context.getCurrentEndpoint(), iter, (Class<T>) obj.getClass());
+    return createDirectPlan(context.getCurrentEndpoint(), Collections.singletonList(obj), (Class<T>) obj.getClass());
 
   }
-  public static <T> PhysicalPlan createDirectPlan(DrillbitEndpoint endpoint, Iterator<T> iterator, Class<T> clazz){
-    PojoRecordReader<T> reader = new PojoRecordReader<T>(clazz, iterator);
+
+  public static <T> PhysicalPlan createDirectPlan(DrillbitEndpoint endpoint, List<T> records, Class<T> clazz){
+    PojoRecordReader<T> reader = new PojoRecordReader<>(clazz, records);
     DirectGroupScan scan = new DirectGroupScan(reader);
     Screen screen = new Screen(scan, endpoint);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFileHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFileHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFileHandler.java
index 5e6af7c..307b01d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFileHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFileHandler.java
@@ -102,6 +102,6 @@ public class ShowFileHandler extends DefaultSqlHandler {
                                                                  fileStatus.getAccessTime(), fileStatus.getModificationTime());
       rows.add(result);
     }
-    return DirectPlan.createDirectPlan(context.getCurrentEndpoint(), rows.iterator(), ShowFilesCommandResult.class);
+    return DirectPlan.createDirectPlan(context.getCurrentEndpoint(), rows, ShowFilesCommandResult.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
new file mode 100644
index 0000000..ccd622b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
@@ -0,0 +1,261 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.drill.exec.store;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ColumnExplorer {
+
+  private final String partitionDesignator;
+  private final List<SchemaPath> columns;
+  private final boolean isStarQuery;
+  private final List<Integer> selectedPartitionColumns;
+  private final List<SchemaPath> tableColumns;
+  private final Map<String, ImplicitFileColumns> allImplicitColumns;
+  private final Map<String, ImplicitFileColumns> selectedImplicitColumns;
+
+
+  /**
+   * Helper class that encapsulates logic for sorting out columns
+   * between actual table columns, partition columns and implicit file columns.
+   * Also populates map with implicit columns names as keys and their values
+   */
+  public ColumnExplorer(FragmentContext context, List<SchemaPath> columns) {
+    this(context.getOptions(), columns);
+  }
+
+  /**
+   * Helper class that encapsulates logic for sorting out columns
+   * between actual table columns, partition columns and implicit file columns.
+   * Also populates map with implicit columns names as keys and their values
+   */
+  public ColumnExplorer(OptionManager optionManager, List<SchemaPath> columns) {
+    this.partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+    this.columns = columns;
+    this.isStarQuery = columns != null && AbstractRecordReader.isStarQuery(columns);
+    this.selectedPartitionColumns = Lists.newArrayList();
+    this.tableColumns = Lists.newArrayList();
+    this.allImplicitColumns = initImplicitFileColumns(optionManager);
+    this.selectedImplicitColumns = CaseInsensitiveMap.newHashMap();
+
+    init();
+  }
+
+  /**
+   * Creates case insensitive map with implicit file columns as keys and appropriate ImplicitFileColumns enum as values
+   */
+  public static Map<String, ImplicitFileColumns> initImplicitFileColumns(OptionManager optionManager) {
+    Map<String, ImplicitFileColumns> map = CaseInsensitiveMap.newHashMap();
+    for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
+      OptionValue optionValue;
+      if ((optionValue = optionManager.getOption(e.name)) != null) {
+        map.put(optionValue.string_val, e);
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Checks if given column is partition or not.
+   *
+   * @param optionManager options
+   * @param column column
+   * @return true if given column is partition, false otherwise
+   */
+  public static boolean isPartitionColumn(OptionManager optionManager, SchemaPath column){
+    String partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+    String path = column.getAsUnescapedPath();
+    return isPartitionColumn(partitionDesignator, path);
+  }
+
+  /**
+   * Checks if given column is partition or not.
+   *
+   * @param partitionDesignator partition designator
+   * @param path column path
+   * @return true if given column is partition, false otherwise
+   */
+  public static boolean isPartitionColumn(String partitionDesignator, String path){
+    Pattern pattern = Pattern.compile(String.format("%s[0-9]+", partitionDesignator));
+    Matcher matcher = pattern.matcher(path);
+    return matcher.matches();
+  }
+
+  /**
+   * Compares selection root and actual file path to determine partition columns values.
+   * Adds implicit file columns according to columns list.
+   *
+   * @return map with columns names as keys and their values
+   */
+  public Map<String, String> populateImplicitColumns(FileWork work, String selectionRoot) {
+    return populateImplicitColumns(work.getPath(), selectionRoot);
+  }
+
+  /**
+   * Compares selection root and actual file path to determine partition columns values.
+   * Adds implicit file columns according to columns list.
+   *
+   * @return map with columns names as keys and their values
+   */
+  public Map<String, String> populateImplicitColumns(String filePath, String selectionRoot) {
+    Map<String, String> implicitValues = Maps.newLinkedHashMap();
+    if (selectionRoot != null) {
+      String[] r = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString().split("/");
+      Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
+      String[] p = path.toString().split("/");
+      if (p.length > r.length) {
+        String[] q = ArrayUtils.subarray(p, r.length, p.length - 1);
+        for (int a = 0; a < q.length; a++) {
+          if (isStarQuery || selectedPartitionColumns.contains(a)) {
+            implicitValues.put(partitionDesignator + a, q[a]);
+          }
+        }
+      }
+      //add implicit file columns
+      for (Map.Entry<String, ImplicitFileColumns> entry : selectedImplicitColumns.entrySet()) {
+        implicitValues.put(entry.getKey(), entry.getValue().getValue(path));
+      }
+    }
+    return implicitValues;
+  }
+
+  public boolean isStarQuery() {
+    return isStarQuery;
+  }
+
+  public List<SchemaPath> getTableColumns() {
+    return tableColumns;
+  }
+
+  /**
+   * Checks if current column selection contains partition columns.
+   *
+   * @return true if partition columns are present, false otherwise
+   */
+  public boolean containsPartitionColumns() {
+    return !selectedPartitionColumns.isEmpty();
+  }
+
+  /**
+   * Checks if current column selection contains implicit columns.
+   *
+   * @return true if implicit columns are present, false otherwise
+   */
+  public boolean containsImplicitColumns() {
+    return !selectedImplicitColumns.isEmpty();
+  }
+
+  /**
+   * If it is not star query, sorts out columns into three categories:
+   * 1. table columns
+   * 2. partition columns
+   * 3. implicit file columns
+   */
+  private void init() {
+    if (isStarQuery) {
+      selectedImplicitColumns.putAll(allImplicitColumns);
+    } else {
+      for (SchemaPath column : columns) {
+        String path = column.getAsUnescapedPath();
+        if (isPartitionColumn(partitionDesignator, path)) {
+          selectedPartitionColumns.add(Integer.parseInt(path.substring(partitionDesignator.length())));
+        } else if (allImplicitColumns.get(path) != null) {
+          selectedImplicitColumns.put(path, allImplicitColumns.get(path));
+        } else {
+          tableColumns.add(column);
+        }
+      }
+    }
+  }
+
+  /**
+   * Columns that give information from where file data comes from.
+   * Columns are implicit, so should be called explicitly in query
+   */
+  public enum ImplicitFileColumns {
+
+    /**
+     * Fully qualified name, contains full path to file and file name
+     */
+    FQN (ExecConstants.IMPLICIT_FQN_COLUMN_LABEL) {
+      @Override
+      public String getValue(Path path) {
+        return path.toUri().getPath();
+      }
+    },
+
+    /**
+     * Full path to file without file name
+     */
+    FILEPATH (ExecConstants.IMPLICIT_FILEPATH_COLUMN_LABEL) {
+      @Override
+      public String getValue(Path path) {
+        return path.getParent().toUri().getPath();
+      }
+    },
+
+    /**
+     * File name with extension without path
+     */
+    FILENAME (ExecConstants.IMPLICIT_FILENAME_COLUMN_LABEL) {
+      @Override
+      public String getValue(Path path) {
+        return path.getName();
+      }
+    },
+
+    /**
+     * File suffix (without dot at the beginning)
+     */
+    SUFFIX (ExecConstants.IMPLICIT_SUFFIX_COLUMN_LABEL) {
+      @Override
+      public String getValue(Path path) {
+        return Files.getFileExtension(path.getName());
+      }
+    };
+
+    String name;
+
+    ImplicitFileColumns(String name) {
+      this.name = name;
+    }
+
+    /**
+     * Using file path calculates value for each implicit file column
+     */
+    public abstract String getValue(Path path);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
deleted file mode 100644
index 42ff827..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ImplicitColumnExplorer.java
+++ /dev/null
@@ -1,219 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.drill.exec.store;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.map.CaseInsensitiveMap;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.store.dfs.easy.FileWork;
-import org.apache.hadoop.fs.Path;
-
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class ImplicitColumnExplorer {
-
-  private final String partitionDesignator;
-  private final List<SchemaPath> columns;
-  private final boolean isStarQuery;
-  private final List<Integer> selectedPartitionColumns;
-  private final List<SchemaPath> tableColumns;
-  private final Map<String, ImplicitFileColumns> allImplicitColumns;
-  private final Map<String, ImplicitFileColumns> selectedImplicitColumns;
-
-
-  /**
-   * Helper class that encapsulates logic for sorting out columns
-   * between actual table columns, partition columns and implicit file columns.
-   * Also populates map with implicit columns names as keys and their values
-   */
-  public ImplicitColumnExplorer(FragmentContext context, List<SchemaPath> columns) {
-    this(context.getOptions(), columns);
-  }
-
-  /**
-   * Helper class that encapsulates logic for sorting out columns
-   * between actual table columns, partition columns and implicit file columns.
-   * Also populates map with implicit columns names as keys and their values
-   */
-  public ImplicitColumnExplorer(OptionManager optionManager, List<SchemaPath> columns) {
-    this.partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
-    this.columns = columns;
-    this.isStarQuery = columns != null && AbstractRecordReader.isStarQuery(columns);
-    this.selectedPartitionColumns = Lists.newArrayList();
-    this.tableColumns = Lists.newArrayList();
-    this.allImplicitColumns = initImplicitFileColumns(optionManager);
-    this.selectedImplicitColumns = CaseInsensitiveMap.newHashMap();
-
-    init();
-  }
-
-  /**
-   * Creates case insensitive map with implicit file columns as keys and appropriate ImplicitFileColumns enum as values
-   */
-  public static Map<String, ImplicitFileColumns> initImplicitFileColumns(OptionManager optionManager) {
-    Map<String, ImplicitFileColumns> map = CaseInsensitiveMap.newHashMap();
-    for (ImplicitFileColumns e : ImplicitFileColumns.values()) {
-      OptionValue optionValue;
-      if ((optionValue = optionManager.getOption(e.name)) != null) {
-        map.put(optionValue.string_val, e);
-      }
-    }
-    return map;
-  }
-
-  /**
-   * Compares selection root and actual file path to determine partition columns values.
-   * Adds implicit file columns according to columns list.
-   *
-   * @return map with columns names as keys and their values
-   */
-  public Map<String, String> populateImplicitColumns(FileWork work, String selectionRoot) {
-    return populateImplicitColumns(work.getPath(), selectionRoot);
-  }
-
-  /**
-   * Compares selection root and actual file path to determine partition columns values.
-   * Adds implicit file columns according to columns list.
-   *
-   * @return map with columns names as keys and their values
-   */
-  public Map<String, String> populateImplicitColumns(String filePath, String selectionRoot) {
-    Map<String, String> implicitValues = Maps.newLinkedHashMap();
-    if (selectionRoot != null) {
-      String[] r = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString().split("/");
-      Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
-      String[] p = path.toString().split("/");
-      if (p.length > r.length) {
-        String[] q = ArrayUtils.subarray(p, r.length, p.length - 1);
-        for (int a = 0; a < q.length; a++) {
-          if (isStarQuery || selectedPartitionColumns.contains(a)) {
-            implicitValues.put(partitionDesignator + a, q[a]);
-          }
-        }
-      }
-      //add implicit file columns
-      for (Map.Entry<String, ImplicitFileColumns> entry : selectedImplicitColumns.entrySet()) {
-        implicitValues.put(entry.getKey(), entry.getValue().getValue(path));
-      }
-    }
-    return implicitValues;
-  }
-
-  public boolean isStarQuery() {
-    return isStarQuery;
-  }
-
-  public List<SchemaPath> getTableColumns() {
-    return tableColumns;
-  }
-
-  /**
-   * If it is not star query, sorts out columns into three categories:
-   * 1. table columns
-   * 2. partition columns
-   * 3. implicit file columns
-   */
-  private void init() {
-    if (isStarQuery) {
-      selectedImplicitColumns.putAll(allImplicitColumns);
-    } else {
-      Pattern pattern = Pattern.compile(String.format("%s[0-9]+", partitionDesignator));
-      for (SchemaPath column : columns) {
-        String path = column.getAsUnescapedPath();
-        Matcher m = pattern.matcher(path);
-        if (m.matches()) {
-          selectedPartitionColumns.add(Integer.parseInt(path.substring(partitionDesignator.length())));
-        } else if (allImplicitColumns.get(path) != null) {
-          selectedImplicitColumns.put(path, allImplicitColumns.get(path));
-        } else {
-          tableColumns.add(column);
-        }
-      }
-    }
-  }
-
-  /**
-   * Columns that give information from where file data comes from.
-   * Columns are implicit, so should be called explicitly in query
-   */
-  public enum ImplicitFileColumns {
-
-    /**
-     * Fully qualified name, contains full path to file and file name
-     */
-    FQN (ExecConstants.IMPLICIT_FQN_COLUMN_LABEL) {
-      @Override
-      public String getValue(Path path) {
-        return path.toString();
-      }
-    },
-
-    /**
-     * Full path to file without file name
-     */
-    FILEPATH (ExecConstants.IMPLICIT_FILEPATH_COLUMN_LABEL) {
-      @Override
-      public String getValue(Path path) {
-        return path.getParent().toString();
-      }
-    },
-
-    /**
-     * File name with extension without path
-     */
-    FILENAME (ExecConstants.IMPLICIT_FILENAME_COLUMN_LABEL) {
-      @Override
-      public String getValue(Path path) {
-        return path.getName();
-      }
-    },
-
-    /**
-     * File suffix (without dot at the beginning)
-     */
-    SUFFIX (ExecConstants.IMPLICIT_SUFFIX_COLUMN_LABEL) {
-      @Override
-      public String getValue(Path path) {
-        return Files.getFileExtension(path.getName());
-      }
-    };
-
-    String name;
-
-    ImplicitFileColumns(String name) {
-      this.name = name;
-    }
-
-    /**
-     * Using file path calculates value for each implicit file column
-     */
-    public abstract String getValue(Path path);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 776d806..1f7bce9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -41,7 +41,7 @@ import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.ImplicitColumnExplorer;
+import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.RecordWriter;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
@@ -52,7 +52,6 @@ import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
@@ -128,7 +127,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
 
   @SuppressWarnings("resource")
   CloseableRecordBatch getReaderBatch(FragmentContext context, EasySubScan scan) throws ExecutionSetupException {
-    final ImplicitColumnExplorer columnExplorer = new ImplicitColumnExplorer(context, scan.getColumns());
+    final ColumnExplorer columnExplorer = new ColumnExplorer(context, scan.getColumns());
 
     if (!columnExplorer.isStarQuery()) {
       scan = new EasySubScan(scan.getUserName(), scan.getWorkUnits(), scan.getFormatPlugin(),

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
index a4b2fad..67b2e5c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -33,10 +33,9 @@ import java.util.List;
 
 @JsonTypeName("direct-scan")
 public class DirectGroupScan extends AbstractGroupScan {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectGroupScan.class);
 
-  private final RecordReader reader;
-  private final ScanStats stats;
+  protected final RecordReader reader;
+  protected final ScanStats stats;
 
   public DirectGroupScan(RecordReader reader) {
     this(reader, ScanStats.TRIVIAL_TABLE);

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
new file mode 100644
index 0000000..505d68e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
@@ -0,0 +1,86 @@
+/*
+ * 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.drill.exec.store.direct;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.store.RecordReader;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Represents direct scan based on metadata information.
+ * For example, for parquet files it can be obtained from parquet footer (total row count)
+ * or from parquet metadata files (column counts).
+ * Contains reader, statistics and list of scanned files if present.
+ */
+@JsonTypeName("metadata-direct-scan")
+public class MetadataDirectGroupScan extends DirectGroupScan {
+
+  private final Collection<String> files;
+
+  public MetadataDirectGroupScan(RecordReader reader, Collection<String> files) {
+    super(reader);
+    this.files = files;
+  }
+
+  public MetadataDirectGroupScan(RecordReader reader, Collection<String> files, ScanStats stats) {
+    super(reader, stats);
+    this.files = files;
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+    assert children == null || children.isEmpty();
+    return new MetadataDirectGroupScan(reader, files, stats);
+  }
+
+  @Override
+  public GroupScan clone(List<SchemaPath> columns) {
+    return this;
+  }
+
+  /**
+   * <p>
+   * Returns string representation of group scan data.
+   * Includes list of files if present.
+   * </p>
+   *
+   * <p>
+   * Example: [files = [/tmp/0_0_0.parquet], numFiles = 1]
+   * </p>
+   *
+   * @return string representation of group scan data
+   */
+  @Override
+  public String getDigest() {
+    if (files != null) {
+      StringBuilder builder = new StringBuilder();
+      builder.append("files = ").append(files).append(", ");
+      builder.append("numFiles = ").append(files.size()).append(", ");
+      return builder.append(super.getDigest()).toString();
+    }
+    return super.getDigest();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
index aee3dc1..e96ec68 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -250,7 +250,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
 
     @Override
     public PojoRecordReader<Records.Catalog> getRecordReader() {
-      return new PojoRecordReader<>(Records.Catalog.class, records.iterator());
+      return new PojoRecordReader<>(Records.Catalog.class, records);
     }
 
     @Override
@@ -269,7 +269,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
 
     @Override
     public PojoRecordReader<Records.Schema> getRecordReader() {
-      return new PojoRecordReader<>(Records.Schema.class, records.iterator());
+      return new PojoRecordReader<>(Records.Schema.class, records);
     }
 
     @Override
@@ -290,7 +290,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
 
     @Override
     public PojoRecordReader<Records.Table> getRecordReader() {
-      return new PojoRecordReader<>(Records.Table.class, records.iterator());
+      return new PojoRecordReader<>(Records.Table.class, records);
     }
 
     @Override
@@ -341,7 +341,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
 
     @Override
     public PojoRecordReader<Records.View> getRecordReader() {
-      return new PojoRecordReader<>(Records.View.class, records.iterator());
+      return new PojoRecordReader<>(Records.View.class, records);
     }
 
     @Override
@@ -362,7 +362,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
 
     @Override
     public PojoRecordReader<Records.Column> getRecordReader() {
-      return new PojoRecordReader<>(Records.Column.class, records.iterator());
+      return new PojoRecordReader<>(Records.Column.class, records);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 30f607d..c333a3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -54,7 +54,7 @@ import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.store.ImplicitColumnExplorer;
+import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FileSelection;
@@ -1063,7 +1063,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     ParquetFilterPredicate filterPredicate = null;
 
     for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) {
-      final ImplicitColumnExplorer columnExplorer = new ImplicitColumnExplorer(optionManager, this.columns);
+      final ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, this.columns);
       Map<String, String> implicitColValues = columnExplorer.populateImplicitColumns(file.getPath(), selectionRoot);
 
       for (RowGroupMetadata rowGroup : file.getRowGroups()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index 5e22458..490a5a0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -32,7 +32,7 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.store.ImplicitColumnExplorer;
+import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
@@ -63,7 +63,7 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
     Preconditions.checkArgument(children.isEmpty());
     OperatorContext oContext = context.newOperatorContext(rowGroupScan);
 
-    final ImplicitColumnExplorer columnExplorer = new ImplicitColumnExplorer(context, rowGroupScan.getColumns());
+    final ColumnExplorer columnExplorer = new ColumnExplorer(context, rowGroupScan.getColumns());
 
     if (!columnExplorer.isStarQuery()) {
       rowGroupScan = new ParquetRowGroupScan(rowGroupScan.getUserName(), rowGroupScan.getStorageEngine(),

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoRecordReader.java
new file mode 100644
index 0000000..0c1144a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoRecordReader.java
@@ -0,0 +1,157 @@
+/*
+ * 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.drill.exec.store.pojo;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Parent class for all pojo readers. Pojo readers can be based on java class (field list is predefined) or dynamic.
+ * Contains general logic for initiating writers and reading values from each row fields.
+ */
+public abstract class AbstractPojoRecordReader<T> extends AbstractRecordReader implements Iterable<T> {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPojoRecordReader.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(AbstractPojoRecordReader.class);
+
+  protected final List<T> records;
+  protected List<PojoWriter> writers;
+
+  private Iterator<T> currentIterator;
+  private OperatorContext operatorContext;
+
+  protected AbstractPojoRecordReader(List<T> records) {
+    this.records = records;
+  }
+
+  @Override
+  public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+    operatorContext = context;
+    writers = setupWriters(output);
+    currentIterator = records.iterator();
+  }
+
+  @Override
+  public int next() {
+    boolean allocated = false;
+    injector.injectPause(operatorContext.getExecutionControls(), "read-next", logger);
+
+    int recordCount = 0;
+    while (currentIterator.hasNext()) {
+      if (!allocated) {
+        allocate();
+        allocated = true;
+      }
+
+      T row = currentIterator.next();
+      for (int i = 0; i < writers.size(); i++) {
+        PojoWriter writer = writers.get(i);
+        writer.writeField(getFieldValue(row, i), recordCount);
+      }
+      recordCount++;
+    }
+
+    if (recordCount != 0) {
+      setValueCount(recordCount);
+    }
+    return recordCount;
+  }
+
+  @Override
+  public void allocate(Map<String, ValueVector> vectorMap) throws OutOfMemoryException {
+    for (final ValueVector v : vectorMap.values()) {
+      AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10);
+    }
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return records.iterator();
+  }
+
+  /**
+   * Creates writer based input class type and then initiates it.
+   *
+   * @param type class type
+   * @param fieldName field name
+   * @param output output mutator
+   * @return pojo writer
+   */
+  protected PojoWriter initWriter(Class<?> type, String fieldName, OutputMutator output) throws ExecutionSetupException {
+    PojoWriter writer = PojoWriters.getWriter(type, fieldName, output.getManagedBuffer());
+    try {
+      writer.init(output);
+      return writer;
+    } catch (SchemaChangeException e) {
+      throw new ExecutionSetupException("Failure while setting up schema for AbstractPojoRecordReader.", e);
+    }
+  }
+
+  /**
+   * Allocates buffers for each writer.
+   */
+  private void allocate() {
+    for (PojoWriter writer : writers) {
+      writer.allocate();
+    }
+  }
+
+  /**
+   * Sets number of written records for each writer.
+   *
+   * @param recordCount number of records written
+   */
+  private void setValueCount(int recordCount) {
+    for (PojoWriter writer : writers) {
+      writer.setValueCount(recordCount);
+    }
+  }
+
+  /**
+   * Setups writers for each field in the row.
+   *
+   * @param output output mutator
+   * @return list of pojo writers
+   */
+  protected abstract List<PojoWriter> setupWriters(OutputMutator output) throws ExecutionSetupException;
+
+  /**
+   * Retrieves field value to be written based for given row and field position.
+   *
+   * @param row current row
+   * @param fieldPosition current field position
+   * @return field value to be written for given row
+   */
+  protected abstract Object getFieldValue(T row, int fieldPosition);
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoWriter.java
new file mode 100644
index 0000000..a2a4644
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractPojoWriter.java
@@ -0,0 +1,64 @@
+/*
+ * 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.drill.exec.store.pojo;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Parent class for all pojo writers created for each field.
+ * Contains common logic for initializing value vector, stores field name and its type.
+ */
+public abstract class AbstractPojoWriter<V extends ValueVector> implements PojoWriter {
+
+  protected V vector;
+  private final String fieldName;
+  private final MajorType type;
+
+  public AbstractPojoWriter(String fieldName, MajorType type) {
+    this.fieldName = fieldName;
+    this.type = type;
+  }
+
+  @Override
+  public void init(OutputMutator output) throws SchemaChangeException {
+    MaterializedField mf = MaterializedField.create(fieldName, type);
+    @SuppressWarnings("unchecked")
+    Class<V> valueVectorClass = (Class<V>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode());
+    this.vector = output.addField(mf, valueVectorClass);
+  }
+
+  @Override
+  public void allocate() {
+    vector.allocateNew();
+  }
+
+  @Override
+  public void setValueCount(int valueCount) {
+    vector.getMutator().setValueCount(valueCount);
+  }
+
+  @Override
+  public void cleanup() {
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
deleted file mode 100644
index c41a07a..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
+++ /dev/null
@@ -1,62 +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.drill.exec.store.pojo;
-
-import java.lang.reflect.Field;
-
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.vector.ValueVector;
-
-abstract class AbstractWriter<V extends ValueVector> implements PojoWriter{
-
-  protected final Field field;
-  protected V vector;
-  protected final MajorType type;
-
-  public AbstractWriter(Field field, MajorType type){
-    this.field = field;
-    this.type = type;
-  }
-
-  @Override
-  public void init(OutputMutator output) throws SchemaChangeException {
-    MaterializedField mf = MaterializedField.create(field.getName(), type);
-    @SuppressWarnings("unchecked")
-    Class<V> valueVectorClass = (Class<V>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode());
-    this.vector = output.addField(mf, valueVectorClass);
-  }
-
-  @Override
-  public void allocate() {
-    vector.allocateNew();
-  }
-
-  public void setValueCount(int valueCount){
-    vector.getMutator().setValueCount(valueCount);
-  }
-
-  @Override
-  public void cleanup() {
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/DynamicPojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/DynamicPojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/DynamicPojoRecordReader.java
new file mode 100644
index 0000000..82383f0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/DynamicPojoRecordReader.java
@@ -0,0 +1,71 @@
+/*
+ * 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.drill.exec.store.pojo;
+
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Dynamically reads values from the given list of records.
+ * Creates writers based on given schema.
+ *
+ * @param <T> type of given values, if contains various types, use Object class
+ */
+public class DynamicPojoRecordReader<T> extends AbstractPojoRecordReader<List<T>> {
+
+  private final LinkedHashMap<String, Class<?>> schema;
+
+  public DynamicPojoRecordReader(LinkedHashMap<String, Class<?>> schema, List<List<T>> records) {
+    super(records);
+    Preconditions.checkState(schema != null && !schema.isEmpty(), "Undefined schema is not allowed.");
+    this.schema = schema;
+  }
+
+  /**
+   * Initiates writers based on given schema which contains field name and its type.
+   *
+   * @param output output mutator
+   * @return list of pojo writers
+   */
+  @Override
+  protected List<PojoWriter> setupWriters(OutputMutator output) throws ExecutionSetupException {
+    List<PojoWriter> writers = new ArrayList<>();
+    for (Map.Entry<String, Class<?>> field : schema.entrySet()) {
+      writers.add(initWriter(field.getValue(), field.getKey(), output));
+    }
+    return writers;
+  }
+
+  @Override
+  protected Object getFieldValue(List<T> row, int fieldPosition) {
+    return row.get(fieldPosition);
+  }
+
+  @Override
+  public String toString() {
+    return "DynamicPojoRecordReader{" +
+        "records = " + records +
+        "}";
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
index baf07a4..c3b6883 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,173 +17,66 @@
  */
 package org.apache.drill.exec.store.pojo;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.sql.Timestamp;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.pojo.Writers.BitWriter;
-import org.apache.drill.exec.store.pojo.Writers.DoubleWriter;
-import org.apache.drill.exec.store.pojo.Writers.EnumWriter;
-import org.apache.drill.exec.store.pojo.Writers.IntWriter;
-import org.apache.drill.exec.store.pojo.Writers.LongWriter;
-import org.apache.drill.exec.store.pojo.Writers.NBigIntWriter;
-import org.apache.drill.exec.store.pojo.Writers.NBooleanWriter;
-import org.apache.drill.exec.store.pojo.Writers.NDoubleWriter;
-import org.apache.drill.exec.store.pojo.Writers.NIntWriter;
-import org.apache.drill.exec.store.pojo.Writers.NTimeStampWriter;
-import org.apache.drill.exec.store.pojo.Writers.StringWriter;
-import org.apache.drill.exec.testing.ControlsInjector;
-import org.apache.drill.exec.testing.ControlsInjectorFactory;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.ValueVector;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.List;
 
-public class PojoRecordReader<T> extends AbstractRecordReader implements Iterable<T> {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoRecordReader.class);
-  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(PojoRecordReader.class);
+/**
+ * Reads values from the given list of pojo instances.
+ * Fields writers are determined based on pojo field class types.
+ *
+ * @param <T> pojo class type
+ */
+public class PojoRecordReader<T> extends AbstractPojoRecordReader<T> {
 
   private final Class<T> pojoClass;
-  private final List<T> pojoObjects;
-  private PojoWriter[] writers;
-  private boolean doCurrent;
-  private T currentPojo;
-  private OperatorContext operatorContext;
+  private final List<Field> fields;
 
-  private Iterator<T> currentIterator;
-
-  /**
-   * TODO: Cleanup the callers to pass the List of POJO objects directly rather than iterator.
-   * @param pojoClass
-   * @param iterator
-   */
-  public PojoRecordReader(Class<T> pojoClass, Iterator<T> iterator) {
+  public PojoRecordReader(Class<T> pojoClass, List<T> records) {
+    super(records);
     this.pojoClass = pojoClass;
-    this.pojoObjects = ImmutableList.copyOf(iterator);
+    this.fields = new ArrayList<>();
   }
 
+  /**
+   * Creates writers based on pojo field class types. Ignores static fields.
+   *
+   * @param output output mutator
+   * @return list of pojo writers
+   */
   @Override
-  public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
-    operatorContext = context;
-    try {
-      Field[] fields = pojoClass.getDeclaredFields();
-      List<PojoWriter> writers = Lists.newArrayList();
-
-      for (int i = 0; i < fields.length; i++) {
-        Field f = fields[i];
-
-        if (Modifier.isStatic(f.getModifiers())) {
-          continue;
-        }
-
-        Class<?> type = f.getType();
-        PojoWriter w = null;
-        if(type == int.class) {
-          w = new IntWriter(f);
-        } else if(type == Integer.class) {
-          w = new NIntWriter(f);
-        } else if(type == Long.class) {
-          w = new NBigIntWriter(f);
-        } else if(type == Boolean.class) {
-          w = new NBooleanWriter(f);
-        } else if(type == double.class) {
-          w = new DoubleWriter(f);
-        } else if(type == Double.class) {
-          w = new NDoubleWriter(f);
-        } else if(type.isEnum()) {
-          w = new EnumWriter(f, output.getManagedBuffer());
-        } else if(type == boolean.class) {
-          w = new BitWriter(f);
-        } else if(type == long.class) {
-          w = new LongWriter(f);
-        } else if(type == String.class) {
-          w = new StringWriter(f, output.getManagedBuffer());
-        } else if (type == Timestamp.class) {
-          w = new NTimeStampWriter(f);
-        } else {
-          throw new ExecutionSetupException(String.format("PojoRecord reader doesn't yet support conversions from type [%s].", type));
-        }
-        writers.add(w);
-        w.init(output);
+  protected List<PojoWriter> setupWriters(OutputMutator output) throws ExecutionSetupException {
+    List<PojoWriter> writers = new ArrayList<>();
+    Field[] declaredFields = pojoClass.getDeclaredFields();
+    for (Field field : declaredFields) {
+      if (Modifier.isStatic(field.getModifiers())) {
+        continue;
       }
-
-      this.writers = writers.toArray(new PojoWriter[writers.size()]);
-
-    } catch(SchemaChangeException e) {
-      throw new ExecutionSetupException("Failure while setting up schema for PojoRecordReader.", e);
-    }
-
-    currentIterator = pojoObjects.iterator();
-  }
-
-  @Override
-  public void allocate(Map<String, ValueVector> vectorMap) throws OutOfMemoryException {
-    for (final ValueVector v : vectorMap.values()) {
-      AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10);
-    }
-  }
-
-  private void allocate() {
-    for (PojoWriter writer : writers) {
-      writer.allocate();
-    }
-  }
-
-  private void setValueCount(int i) {
-    for (PojoWriter writer : writers) {
-      writer.setValueCount(i);
+      writers.add(initWriter(field.getType(), field.getName(), output));
+      fields.add(field);
     }
+    return writers;
   }
 
   @Override
-  public int next() {
-    boolean allocated = false;
-    injector.injectPause(operatorContext.getExecutionControls(), "read-next", logger);
+  protected Object getFieldValue(T row, int fieldPosition) {
     try {
-      int i =0;
-      while (doCurrent || currentIterator.hasNext()) {
-        if (doCurrent) {
-          doCurrent = false;
-        } else {
-          currentPojo = currentIterator.next();
-        }
-
-        if (!allocated) {
-          allocate();
-          allocated = true;
-        }
-
-        for (PojoWriter writer : writers) {
-          writer.writeField(currentPojo, i);
-        }
-        i++;
-      }
-
-      if (i != 0 ) {
-        setValueCount(i);
-      }
-      return i;
+      return fields.get(fieldPosition).get(row);
     } catch (IllegalArgumentException | IllegalAccessException e) {
-      throw new RuntimeException("Failure while trying to use PojoRecordReader.", e);
+      throw new DrillRuntimeException("Failure while trying to use PojoRecordReader.", e);
     }
   }
 
   @Override
-  public Iterator<T> iterator() {
-    return pojoObjects.iterator();
-  }
-
-  @Override
-  public void close() {
+  public String toString() {
+    return "PojoRecordReader{" +
+        "pojoClass = " + pojoClass +
+        ", recordCount = " + records.size() +
+        "}";
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/8b564235/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
index 31748f4..335bfb1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,10 +20,40 @@ package org.apache.drill.exec.store.pojo;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 
-interface PojoWriter{
-  void writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException ;
+/**
+ * Pojo writer interface for writers based on types supported for pojo.
+ */
+public interface PojoWriter {
+
+  /**
+   * Writes given value to the given position of the bit to set.
+   *
+   * @param value values to be written
+   * @param outboundIndex position of the bit
+   */
+  void writeField(Object value, int outboundIndex);
+
+  /**
+   * Initializes value vector.
+   *
+   * @param output output mutator
+   */
   void init(OutputMutator output) throws SchemaChangeException;
+
+  /**
+   * Allocates new buffer for value vector.
+   */
   void allocate();
-  void setValueCount(int i);
+
+  /**
+   * Sets number of written records.
+   *
+   * @param recordCount record count
+   */
+  void setValueCount(int recordCount);
+
+  /**
+   * Performs clean up if needed.
+   */
   void cleanup();
 }
\ No newline at end of file


[02/13] drill git commit: DRILL-5601: Rollup of external sort fixes an improvements

Posted by jn...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
index 213720f..cd03b70 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
@@ -19,7 +19,160 @@ package org.apache.drill.exec.physical.impl.xsort.managed;
 
 import com.google.common.annotations.VisibleForTesting;
 
+/**
+ * Computes the memory needs for input batches, spill batches and merge
+ * batches. The key challenges that this code tries to overcome are:
+ * <ul>
+ * <li>Drill is not designed for the small memory allocations,
+ * but the planner may provide such allocations because the memory per
+ * query is divided among slices (minor fragments) and among buffering
+ * operators, leaving very little per operator.</li>
+ * <li>Drill does not provide the detailed memory information needed to
+ * carefully manage memory in tight constraints.</li>
+ * <li>But, Drill has a death penalty for going over the memory limit.</li>
+ * </ul>
+ * As a result, this class is a bit of a hack: it attempt to consider a
+ * number of ill-defined factors in order to divide up memory use in a
+ * way that prevents OOM errors.
+ * <p>
+ * First, it is necessary to differentiate two concepts:
+ * <ul>
+ * <li>The <i>data size</i> of a batch: the amount of memory needed to hold
+ * the data itself. The data size is constant for any given batch.</li>
+ * <li>The <i>buffer size</i> of the buffers that hold the data. The buffer
+ * size varies wildly depending on how the batch was produced.</li>
+ * </ul>
+ * The three kinds of buffer layouts seen to date include:
+ * <ul>
+ * <li>One buffer per vector component (data, offsets, null flags, etc.)
+ * &ndash; create by readers, project and other operators.</li>
+ * <li>One buffer for the entire batch, with each vector component using
+ * a slice of the overall buffer. &ndash; case for batches deserialized from
+ * exchanges.</li>
+ * <li>One buffer for each top-level vector, with component vectors
+ * using slices of the overall vector buffer &ndash; the result of reading
+ * spilled batches from disk.</li>
+ * </ul>
+ * In each case, buffer sizes are power-of-two rounded from the data size.
+ * But since the data is grouped differently in each case, the resulting buffer
+ * sizes vary considerably.
+ * <p>
+ * As a result, we can never be sure of the amount of memory needed for a
+ * batch. So, we have to estimate based on a number of factors:
+ * <ul>
+ * <li>Uses the {@link RecordBatchSizer} to estimate the data size and
+ * buffer size of each incoming batch.</li>
+ * <li>Estimates the internal fragmentation due to power-of-two rounding.</li>
+ * <li>Configured preferences for spill and output batches.</li>
+ * </ul>
+ * The code handles "normal" and "low" memory conditions.
+ * <ul>
+ * <li>In normal memory, we simply work out the number of preferred-size
+ * batches that fit in memory (based on the predicted buffer size.)</li>
+ * <li>In low memory, we divide up the available memory to produce the
+ * spill and merge batch sizes. The sizes will be less than the configured
+ * preference.</li>
+ * </ul>
+ * <p>
+ * The sort has two key configured parameters: the spill file size and the
+ * size of the output (downstream) batch. The spill file size is chosen to
+ * be large enough to ensure efficient I/O, but not so large as to overwhelm
+ * any one spill directory. The output batch size is chosen to be large enough
+ * to amortize the per-batch overhead over the maximum number of records, but
+ * not so large as to overwhelm downstream operators. Setting these parameters
+ * is a judgment call.
+ * <p>
+ * Under limited memory, the above sizes may be too big for the space available.
+ * For example, the default spill file size is 256 MB. But, if the sort is
+ * only given 50 MB, then spill files will be smaller. The default output batch
+ * size is 16 MB, but if the sort is given only 20 MB, then the output batch must
+ * be smaller. The low memory logic starts with the memory available and works
+ * backwards to figure out spill batch size, output batch size and spill file
+ * size. The sizes will be smaller than optimal, but as large as will fit in
+ * the memory provided.
+ */
+
 public class SortMemoryManager {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
+
+  /**
+   * Estimate for typical internal fragmentation in a buffer due to power-of-two
+   * rounding on vectors.
+   * <p>
+   * <p>
+   * <pre>[____|__$__]</pre>
+   * In the above, the brackets represent the whole vector. The
+   * first half is always full. The $ represents the end of data.
+   * When the first half filled, the second
+   * half was allocated. On average, the second half will be half full.
+   * This means that, on average, 1/4 of the allocated space is
+   * unused (the definition of internal fragmentation.)
+   */
+
+  public static final double INTERNAL_FRAGMENTATION_ESTIMATE = 1.0/4.0;
+
+  /**
+   * Given a buffer, this is the assumed amount of space
+   * available for data. (Adding more will double the buffer
+   * size half the time.)
+   */
+
+  public static final double PAYLOAD_FROM_BUFFER = 1 - INTERNAL_FRAGMENTATION_ESTIMATE;
+
+  /**
+   * Given a data size, this is the multiplier to create the buffer
+   * size estimate. (Note: since we work with aggregate batches, we
+   * cannot simply round up to the next power of two: rounding is done
+   * on a vector-by-vector basis. Here we need to estimate the aggregate
+   * effect of rounding.
+   */
+
+  public static final double BUFFER_FROM_PAYLOAD = 3.0 / 2.0;
+
+  /**
+   * On really bad days, we will add one more byte (or value) to a vector
+   * than fits in a power-of-two sized buffer, forcing a doubling. In this
+   * case, half the resulting buffer is empty.
+   */
+
+  public static final double WORST_CASE_BUFFER_RATIO = 2.0;
+
+  /**
+   * Desperate attempt to keep spill batches from being too small in low memory.
+   * <p>
+   * The number is also used for logging: the system will log a warning if
+   * batches fall below this number which may represent too little memory
+   * allocated for the job at hand. (Queries operate on big data: many records.
+   * Batches with too few records are a probable performance hit. But, what is
+   * too few? It is a judgment call.)
+   */
+
+  public static final int MIN_ROWS_PER_SORT_BATCH = 100;
+  public static final double LOW_MEMORY_MERGE_BATCH_RATIO = 0.25;
+
+  public static class BatchSizeEstimate {
+    int dataSize;
+    int expectedBufferSize;
+    int maxBufferSize;
+
+    public void setFromData(int dataSize) {
+      this.dataSize = dataSize;
+      expectedBufferSize = multiply(dataSize, BUFFER_FROM_PAYLOAD);
+      maxBufferSize = multiply(dataSize, WORST_CASE_BUFFER_RATIO);
+    }
+
+    public void setFromBuffer(int bufferSize) {
+      expectedBufferSize = bufferSize;
+      dataSize = multiply(bufferSize, PAYLOAD_FROM_BUFFER);
+      maxBufferSize = multiply(dataSize, WORST_CASE_BUFFER_RATIO);
+    }
+
+    public void setFromWorstCaseBuffer(int bufferSize) {
+      maxBufferSize = bufferSize;
+      dataSize = multiply(maxBufferSize, 1 / WORST_CASE_BUFFER_RATIO);
+      expectedBufferSize = multiply(dataSize, BUFFER_FROM_PAYLOAD);
+    }
+ }
 
   /**
    * Maximum memory this operator may use. Usually comes from the
@@ -42,13 +195,13 @@ public class SortMemoryManager {
    * value.
    */
 
-  private int expectedMergeBatchSize;
+  private final BatchSizeEstimate mergeBatchSize = new BatchSizeEstimate();
 
   /**
    * Estimate of the input batch size based on the largest batch seen
    * thus far.
    */
-  private int estimatedInputBatchSize;
+  private final BatchSizeEstimate inputBatchSize = new BatchSizeEstimate();
 
   /**
    * Maximum memory level before spilling occurs. That is, we can buffer input
@@ -86,7 +239,7 @@ public class SortMemoryManager {
    * details of the data rows for any particular query.
    */
 
-  private int expectedSpillBatchSize;
+  private final BatchSizeEstimate spillBatchSize = new BatchSizeEstimate();
 
   /**
    * The number of records to add to each output batch sent to the
@@ -97,24 +250,41 @@ public class SortMemoryManager {
 
   private SortConfig config;
 
-  private int estimatedInputSize;
-
   private boolean potentialOverflow;
 
-  public SortMemoryManager(SortConfig config, long memoryLimit) {
+  private boolean isLowMemory;
+
+  private boolean performanceWarning;
+
+  public SortMemoryManager(SortConfig config, long opMemoryLimit) {
     this.config = config;
 
     // The maximum memory this operator can use as set by the
     // operator definition (propagated to the allocator.)
 
-    if (config.maxMemory() > 0) {
-      this.memoryLimit = Math.min(memoryLimit, config.maxMemory());
-    } else {
-      this.memoryLimit = memoryLimit;
-    }
+    final long configMemoryLimit = config.maxMemory();
+    memoryLimit = (configMemoryLimit == 0) ? opMemoryLimit
+                : Math.min(opMemoryLimit, configMemoryLimit);
 
     preferredSpillBatchSize = config.spillBatchSize();;
     preferredMergeBatchSize = config.mergeBatchSize();
+
+    // Initialize the buffer memory limit for the first batch.
+    // Assume 1/2 of (allocated - spill batch size).
+
+    bufferMemoryLimit = (memoryLimit - config.spillBatchSize()) / 2;
+    if (bufferMemoryLimit < 0) {
+      // Bad news: not enough for even the spill batch.
+      // Assume half of memory, will adjust later.
+      bufferMemoryLimit = memoryLimit / 2;
+    }
+
+    if (memoryLimit == opMemoryLimit) {
+      logger.debug("Memory config: Allocator limit = {}", memoryLimit);
+    } else {
+      logger.debug("Memory config: Allocator limit = {}, Configured limit: {}",
+                   opMemoryLimit, memoryLimit);
+    }
   }
 
   /**
@@ -134,36 +304,39 @@ public class SortMemoryManager {
    * phase, and how many spill batches we can merge during the merge
    * phase.
    *
-   * @param batchSize the overall size of the current batch received from
+   * @param batchDataSize the overall size of the current batch received from
    * upstream
    * @param batchRowWidth the average width in bytes (including overhead) of
    * rows in the current input batch
    * @param batchRowCount the number of actual (not filtered) records in
    * that upstream batch
+   * @return true if the estimates changed, false if the previous estimates
+   * remain valid
    */
 
-  public void updateEstimates(int batchSize, int batchRowWidth, int batchRowCount) {
+  public boolean updateEstimates(int batchDataSize, int batchRowWidth, int batchRowCount) {
 
     // The record count should never be zero, but better safe than sorry...
 
     if (batchRowCount == 0) {
-      return; }
+      return false; }
 
 
     // Update input batch estimates.
     // Go no further if nothing changed.
 
-    if (! updateInputEstimates(batchSize, batchRowWidth, batchRowCount)) {
-      return;
+    if (! updateInputEstimates(batchDataSize, batchRowWidth, batchRowCount)) {
+      return false;
     }
 
     updateSpillSettings();
     updateMergeSettings();
     adjustForLowMemory();
     logSettings(batchRowCount);
+    return true;
   }
 
-  private boolean updateInputEstimates(int batchSize, int batchRowWidth, int batchRowCount) {
+  private boolean updateInputEstimates(int batchDataSize, int batchRowWidth, int batchRowCount) {
 
     // The row width may end up as zero if all fields are nulls or some
     // other unusual situation. In this case, assume a width of 10 just
@@ -192,17 +365,13 @@ public class SortMemoryManager {
     // batch. Because we are using the actual observed batch size,
     // the size already includes overhead due to power-of-two rounding.
 
-    long origInputBatchSize = estimatedInputBatchSize;
-    estimatedInputBatchSize = Math.max(estimatedInputBatchSize, batchSize);
-
-    // Estimate the total size of each incoming batch plus sv2. Note that, due
-    // to power-of-two rounding, the allocated sv2 size might be twice the data size.
-
-    estimatedInputSize = estimatedInputBatchSize + 4 * batchRowCount;
+    long origInputBatchSize = inputBatchSize.dataSize;
+    inputBatchSize.setFromData(Math.max(inputBatchSize.dataSize, batchDataSize));
 
     // Return whether anything changed.
 
-    return estimatedRowWidth != origRowEstimate || estimatedInputBatchSize != origInputBatchSize;
+    return estimatedRowWidth > origRowEstimate ||
+           inputBatchSize.dataSize > origInputBatchSize;
   }
 
   /**
@@ -215,18 +384,23 @@ public class SortMemoryManager {
 
     spillBatchRowCount = rowsPerBatch(preferredSpillBatchSize);
 
+    // But, don't allow spill batches to be too small; we pay too
+    // much overhead cost for small row counts.
+
+    spillBatchRowCount = Math.max(spillBatchRowCount, MIN_ROWS_PER_SORT_BATCH);
+
     // Compute the actual spill batch size which may be larger or smaller
-    // than the preferred size depending on the row width. Double the estimated
-    // memory needs to allow for power-of-two rounding.
+    // than the preferred size depending on the row width.
 
-    expectedSpillBatchSize = batchForRows(spillBatchRowCount);
+    spillBatchSize.setFromData(spillBatchRowCount * estimatedRowWidth);
 
     // Determine the minimum memory needed for spilling. Spilling is done just
     // before accepting a spill batch, so we must spill if we don't have room for a
     // (worst case) input batch. To spill, we need room for the spill batch created
-    // by merging the batches already in memory.
+    // by merging the batches already in memory. This is a memory calculation,
+    // so use the buffer size for the spill batch.
 
-    bufferMemoryLimit = memoryLimit - expectedSpillBatchSize;
+    bufferMemoryLimit = memoryLimit - 2 * spillBatchSize.maxBufferSize;
   }
 
   /**
@@ -238,13 +412,21 @@ public class SortMemoryManager {
   private void updateMergeSettings() {
 
     mergeBatchRowCount = rowsPerBatch(preferredMergeBatchSize);
-    expectedMergeBatchSize = batchForRows(mergeBatchRowCount);
+
+    // But, don't allow merge batches to be too small; we pay too
+    // much overhead cost for small row counts.
+
+    mergeBatchRowCount = Math.max(mergeBatchRowCount, MIN_ROWS_PER_SORT_BATCH);
+
+    // Compute the actual merge batch size.
+
+    mergeBatchSize.setFromData(mergeBatchRowCount * estimatedRowWidth);
 
     // The merge memory pool assumes we can spill all input batches. The memory
     // available to hold spill batches for merging is total memory minus the
     // expected output batch size.
 
-    mergeMemoryLimit = memoryLimit - expectedMergeBatchSize;
+    mergeMemoryLimit = memoryLimit - mergeBatchSize.maxBufferSize;
   }
 
   /**
@@ -271,22 +453,27 @@ public class SortMemoryManager {
 
   private void adjustForLowMemory() {
 
-    long loadHeadroom = bufferMemoryLimit - 2 * estimatedInputSize;
-    long mergeHeadroom = mergeMemoryLimit - 2 * expectedSpillBatchSize;
-    if (loadHeadroom >= 0  &&  mergeHeadroom >= 0) {
-      return;
-    }
+    potentialOverflow = false;
+    performanceWarning = false;
 
-    lowMemorySpillBatchSize();
-    lowMemoryMergeBatchSize();
+    // Input batches are assumed to have typical fragmentation. Experience
+    // shows that spilled batches have close to the maximum fragmentation.
+
+    long loadHeadroom = bufferMemoryLimit - 2 * inputBatchSize.expectedBufferSize;
+    long mergeHeadroom = mergeMemoryLimit - 2 * spillBatchSize.maxBufferSize;
+    isLowMemory = (loadHeadroom < 0  |  mergeHeadroom < 0);
+    if (! isLowMemory) {
+      return; }
+
+    lowMemoryInternalBatchSizes();
 
     // Sanity check: if we've been given too little memory to make progress,
     // issue a warning but proceed anyway. Should only occur if something is
     // configured terribly wrong.
 
-    long minNeeds = 2 * estimatedInputSize + expectedSpillBatchSize;
+    long minNeeds = 2 * inputBatchSize.expectedBufferSize + spillBatchSize.maxBufferSize;
     if (minNeeds > memoryLimit) {
-      ExternalSortBatch.logger.warn("Potential memory overflow during load phase! " +
+      logger.warn("Potential memory overflow during load phase! " +
           "Minimum needed = {} bytes, actual available = {} bytes",
           minNeeds, memoryLimit);
       bufferMemoryLimit = 0;
@@ -295,14 +482,36 @@ public class SortMemoryManager {
 
     // Sanity check
 
-    minNeeds = 2 * expectedSpillBatchSize + expectedMergeBatchSize;
+    minNeeds = 2 * spillBatchSize.expectedBufferSize + mergeBatchSize.expectedBufferSize;
     if (minNeeds > memoryLimit) {
-      ExternalSortBatch.logger.warn("Potential memory overflow during merge phase! " +
+      logger.warn("Potential memory overflow during merge phase! " +
           "Minimum needed = {} bytes, actual available = {} bytes",
           minNeeds, memoryLimit);
       mergeMemoryLimit = 0;
       potentialOverflow = true;
     }
+
+    // Performance warning
+
+    if (potentialOverflow) {
+      return;
+    }
+    if (spillBatchSize.dataSize < config.spillBatchSize()  &&
+        spillBatchRowCount < Character.MAX_VALUE) {
+      logger.warn("Potential performance degredation due to low memory. " +
+                  "Preferred spill batch size: {}, actual: {}, rows per batch: {}",
+                  config.spillBatchSize(), spillBatchSize.dataSize,
+                  spillBatchRowCount);
+      performanceWarning = true;
+    }
+    if (mergeBatchSize.dataSize < config.mergeBatchSize()  &&
+        mergeBatchRowCount < Character.MAX_VALUE) {
+      logger.warn("Potential performance degredation due to low memory. " +
+                  "Preferred merge batch size: {}, actual: {}, rows per batch: {}",
+                  config.mergeBatchSize(), mergeBatchSize.dataSize,
+                  mergeBatchRowCount);
+      performanceWarning = true;
+    }
   }
 
   /**
@@ -312,52 +521,66 @@ public class SortMemoryManager {
    * one spill batch to make progress.
    */
 
-  private void lowMemorySpillBatchSize() {
+  private void lowMemoryInternalBatchSizes() {
 
     // The "expected" size is with power-of-two rounding in some vectors.
     // We later work backwards to the row count assuming average internal
     // fragmentation.
 
-    // Must hold two input batches. Use (most of) the rest for the spill batch.
+    // Must hold two input batches. Use half of the rest for the spill batch.
+    // In a really bad case, the number here may be negative. We'll fix
+    // it below.
 
-    expectedSpillBatchSize = (int) (memoryLimit - 2 * estimatedInputSize);
+    int spillBufferSize = (int) (memoryLimit - 2 * inputBatchSize.maxBufferSize) / 2;
 
     // But, in the merge phase, we need two spill batches and one output batch.
     // (Assume that the spill and merge are equal sizes.)
-    // Use 3/4 of memory for each batch (to allow power-of-two rounding:
 
-    expectedSpillBatchSize = (int) Math.min(expectedSpillBatchSize, memoryLimit/3);
+    spillBufferSize = (int) Math.min(spillBufferSize, memoryLimit/4);
 
-    // Never going to happen, but let's ensure we don't somehow create large batches.
+    // Compute the size from the buffer. Assume worst-case
+    // fragmentation (as is typical when reading from the spill file.)
 
-    expectedSpillBatchSize = Math.max(expectedSpillBatchSize, SortConfig.MIN_SPILL_BATCH_SIZE);
+    spillBatchSize.setFromWorstCaseBuffer(spillBufferSize);
 
     // Must hold at least one row to spill. That is, we can make progress if we
     // create spill files that consist of single-record batches.
 
-    expectedSpillBatchSize = Math.max(expectedSpillBatchSize, estimatedRowWidth);
+    int spillDataSize = Math.min(spillBatchSize.dataSize, config.spillBatchSize());
+    spillDataSize = Math.max(spillDataSize, estimatedRowWidth);
+    if (spillDataSize != spillBatchSize.dataSize) {
+      spillBatchSize.setFromData(spillDataSize);
+    }
 
     // Work out the spill batch count needed by the spill code. Allow room for
     // power-of-two rounding.
 
-    spillBatchRowCount = rowsPerBatch(expectedSpillBatchSize);
+    spillBatchRowCount = rowsPerBatch(spillBatchSize.dataSize);
 
     // Finally, figure out when we must spill.
 
-    bufferMemoryLimit = memoryLimit - expectedSpillBatchSize;
-  }
+    bufferMemoryLimit = memoryLimit - 2 * spillBatchSize.maxBufferSize;
+    bufferMemoryLimit = Math.max(bufferMemoryLimit, 0);
 
-  /**
-   * For merge batch, we must hold at least two spill batches and
-   * one output batch.
-   */
+    // Assume two spill batches must be merged (plus safety margin.)
+    // The rest can be give to the merge batch.
+
+    long mergeBufferSize = memoryLimit - 2 * spillBatchSize.maxBufferSize;
+
+    // The above calcs assume that the merge batch size is the same as
+    // the spill batch size (the division by three.)
+    // For merge batch, we must hold at least two spill batches and
+    // one output batch, which is why we assumed 3 spill batches.
 
-  private void lowMemoryMergeBatchSize() {
-    expectedMergeBatchSize = (int) (memoryLimit - 2 * expectedSpillBatchSize);
-    expectedMergeBatchSize = Math.max(expectedMergeBatchSize, SortConfig.MIN_MERGE_BATCH_SIZE);
-    expectedMergeBatchSize = Math.max(expectedMergeBatchSize, estimatedRowWidth);
-    mergeBatchRowCount = rowsPerBatch(expectedMergeBatchSize);
-    mergeMemoryLimit = memoryLimit - expectedMergeBatchSize;
+    mergeBatchSize.setFromBuffer((int) mergeBufferSize);
+    int mergeDataSize = Math.min(mergeBatchSize.dataSize, config.mergeBatchSize());
+    mergeDataSize = Math.max(mergeDataSize, estimatedRowWidth);
+    if (mergeDataSize != mergeBatchSize.dataSize) {
+      mergeBatchSize.setFromData(spillDataSize);
+    }
+
+    mergeBatchRowCount = rowsPerBatch(mergeBatchSize.dataSize);
+    mergeMemoryLimit = Math.max(2 * spillBatchSize.expectedBufferSize, memoryLimit - mergeBatchSize.maxBufferSize);
   }
 
   /**
@@ -367,14 +590,34 @@ public class SortMemoryManager {
 
   private void logSettings(int actualRecordCount) {
 
-    ExternalSortBatch.logger.debug("Input Batch Estimates: record size = {} bytes; input batch = {} bytes, {} records",
-                 estimatedRowWidth, estimatedInputBatchSize, actualRecordCount);
-    ExternalSortBatch.logger.debug("Merge batch size = {} bytes, {} records; spill file size: {} bytes",
-                 expectedSpillBatchSize, spillBatchRowCount, config.spillFileSize());
-    ExternalSortBatch.logger.debug("Output batch size = {} bytes, {} records",
-                 expectedMergeBatchSize, mergeBatchRowCount);
-    ExternalSortBatch.logger.debug("Available memory: {}, buffer memory = {}, merge memory = {}",
+    logger.debug("Input Batch Estimates: record size = {} bytes; net = {} bytes, gross = {}, records = {}",
+                 estimatedRowWidth, inputBatchSize.dataSize,
+                 inputBatchSize.expectedBufferSize, actualRecordCount);
+    logger.debug("Spill batch size: net = {} bytes, gross = {} bytes, records = {}; spill file = {} bytes",
+                 spillBatchSize.dataSize, spillBatchSize.expectedBufferSize,
+                 spillBatchRowCount, config.spillFileSize());
+    logger.debug("Output batch size: net = {} bytes, gross = {} bytes, records = {}",
+                 mergeBatchSize.dataSize, mergeBatchSize.expectedBufferSize,
+                 mergeBatchRowCount);
+    logger.debug("Available memory: {}, buffer memory = {}, merge memory = {}",
                  memoryLimit, bufferMemoryLimit, mergeMemoryLimit);
+
+    // Performance warnings due to low row counts per batch.
+    // Low row counts cause excessive per-batch overhead and hurt
+    // performance.
+
+    if (spillBatchRowCount < MIN_ROWS_PER_SORT_BATCH) {
+      logger.warn("Potential performance degredation due to low memory or large input row. " +
+                  "Preferred spill batch row count: {}, actual: {}",
+                  MIN_ROWS_PER_SORT_BATCH, spillBatchRowCount);
+      performanceWarning = true;
+    }
+    if (mergeBatchRowCount < MIN_ROWS_PER_SORT_BATCH) {
+      logger.warn("Potential performance degredation due to low memory or large input row. " +
+                  "Preferred merge batch row count: {}, actual: {}",
+                  MIN_ROWS_PER_SORT_BATCH, mergeBatchRowCount);
+      performanceWarning = true;
+    }
   }
 
   public enum MergeAction { SPILL, MERGE, NONE }
@@ -389,86 +632,120 @@ public class SortMemoryManager {
     }
   }
 
+  /**
+   * Choose a consolidation option during the merge phase depending on memory
+   * available. Preference is given to moving directly onto merging (with no
+   * additional spilling) when possible. But, if memory pressures don't allow
+   * this, we must spill batches and/or merge on-disk spilled runs, to reduce
+   * the final set of runs to something that can be merged in the available
+   * memory.
+   * <p>
+   * Logic is here (returning an enum) rather than in the merge code to allow
+   * unit testing without actually needing batches in memory.
+   *
+   * @param allocMemory
+   *          amount of memory currently allocated (this class knows the total
+   *          memory available)
+   * @param inMemCount
+   *          number of incoming batches in memory (the number is important, not
+   *          the in-memory size; we get the memory size from
+   *          <tt>allocMemory</tt>)
+   * @param spilledRunsCount
+   *          the number of runs sitting on disk to be merged
+   * @return whether to <tt>SPILL</tt> in-memory batches, whether to
+   *         <tt>MERGE<tt> on-disk batches to create a new, larger run, or whether
+   *         to do nothing (<tt>NONE</tt>) and instead advance to the final merge
+   */
+
   public MergeTask consolidateBatches(long allocMemory, int inMemCount, int spilledRunsCount) {
 
-    // Determine additional memory needed to hold one batch from each
-    // spilled run.
+    assert allocMemory == 0 || inMemCount > 0;
+    assert inMemCount + spilledRunsCount > 0;
 
-    // If the on-disk batches and in-memory batches need more memory than
-    // is available, spill some in-memory batches.
+    // If only one spilled run, then merging is not productive regardless
+    // of memory limits.
 
-    if (inMemCount > 0) {
-      long mergeSize = spilledRunsCount * expectedSpillBatchSize;
-      if (allocMemory + mergeSize > mergeMemoryLimit) {
-        return new MergeTask(MergeAction.SPILL, 0);
-      }
+    if (inMemCount == 0 && spilledRunsCount <= 1) {
+      return new MergeTask(MergeAction.NONE, 0);
     }
 
-    // Maximum batches that fit into available memory.
+    // If memory is above the merge memory limit, then must spill
+    // merge to create room for a merge batch.
 
-    int mergeLimit = (int) ((mergeMemoryLimit - allocMemory) / expectedSpillBatchSize);
+    if (allocMemory > mergeMemoryLimit) {
+      return new MergeTask(MergeAction.SPILL, 0);
+    }
 
-    // Can't merge more than the merge limit.
+    // Determine additional memory needed to hold one batch from each
+    // spilled run.
+
+    // Maximum spill batches that fit into available memory.
 
-    mergeLimit = Math.min(mergeLimit, config.mergeLimit());
+    int memMergeLimit = (int) ((mergeMemoryLimit - allocMemory) /
+                                spillBatchSize.expectedBufferSize);
+    memMergeLimit = Math.max(0, memMergeLimit);
 
-    // How many batches to merge?
+    // If batches are in memory, and we need more memory to merge
+    // them all than is actually available, then spill some in-memory
+    // batches.
+
+    if (inMemCount > 0  &&  memMergeLimit < spilledRunsCount) {
+      return new MergeTask(MergeAction.SPILL, 0);
+    }
 
-    int mergeCount = spilledRunsCount - mergeLimit;
-    if (mergeCount <= 0) {
+    // If all batches fit in memory, then no need for a second-generation
+    // merge/spill.
+
+    memMergeLimit = Math.min(memMergeLimit, config.mergeLimit());
+    int mergeRunCount = spilledRunsCount - memMergeLimit;
+    if (mergeRunCount <= 0) {
       return new MergeTask(MergeAction.NONE, 0);
     }
 
-    // We will merge. This will create yet another spilled
-    // run. Account for that.
+    // We need a second generation load-merge-spill cycle
+    // to reduce the number of spilled runs to a smaller set
+    // that will fit in memory.
+
+    // Merging creates another batch. Include one more run
+    // in the merge to create space for the new run.
+
+    mergeRunCount += 1;
 
-    mergeCount += 1;
+    // Merge only as many batches as fit in memory.
+    // Use all memory for this process; no need to reserve space for a
+    // merge output batch. Assume worst case since we are forced to
+    // accept spilled batches blind: we can't limit reading based on memory
+    // limits. Subtract one to allow for the output spill batch.
+
+    memMergeLimit = (int)(memoryLimit / spillBatchSize.maxBufferSize) - 1;
+    mergeRunCount = Math.min(mergeRunCount, memMergeLimit);
 
     // Must merge at least 2 batches to make progress.
-    // This is the the (at least one) excess plus the allowance
-    // above for the new one.
+    // We know we have at least two because of the check done above.
 
-    // Can't merge more than the limit.
+    mergeRunCount = Math.max(mergeRunCount, 2);
 
-    mergeCount = Math.min(mergeCount, config.mergeLimit());
+    // Can't merge more than the merge limit.
 
-    // Do the merge, then loop to try again in case not
-    // all the target batches spilled in one go.
+    mergeRunCount = Math.min(mergeRunCount, config.mergeLimit());
 
-    return new MergeTask(MergeAction.MERGE, mergeCount);
+    return new MergeTask(MergeAction.MERGE, mergeRunCount);
   }
 
   /**
-   * Compute the number of rows per batch assuming that the batch is
-   * subject to average internal fragmentation due to power-of-two
-   * rounding on vectors.
-   * <p>
-   * <pre>[____|__$__]</pre>
-   * In the above, the brackets represent the whole vector. The
-   * first half is always full. When the first half filled, the second
-   * half was allocated. On average, the second half will be half full.
+   * Compute the number of rows that fit into a given batch data size.
    *
    * @param batchSize expected batch size, including internal fragmentation
    * @return number of rows that fit into the batch
    */
 
   private int rowsPerBatch(int batchSize) {
-    int rowCount = batchSize * 3 / 4 / estimatedRowWidth;
+    int rowCount = batchSize / estimatedRowWidth;
     return Math.max(1, Math.min(rowCount, Character.MAX_VALUE));
   }
 
-  /**
-   * Compute the expected number of rows that fit into a given size
-   * batch, accounting for internal fragmentation due to power-of-two
-   * rounding on vector allocations.
-   *
-   * @param rowCount the desired number of rows in the batch
-   * @return the size of resulting batch, including power-of-two
-   * rounding.
-   */
-
-  private int batchForRows(int rowCount) {
-    return estimatedRowWidth * rowCount * 4 / 3;
+  public static int multiply(int byteSize, double multiplier) {
+    return (int) Math.floor(byteSize * multiplier);
   }
 
   // Must spill if we are below the spill point (the amount of memory
@@ -497,17 +774,21 @@ public class SortMemoryManager {
   @VisibleForTesting
   public int getRowWidth() { return estimatedRowWidth; }
   @VisibleForTesting
-  public int getInputBatchSize() { return estimatedInputBatchSize; }
+  public BatchSizeEstimate getInputBatchSize() { return inputBatchSize; }
   @VisibleForTesting
   public int getPreferredSpillBatchSize() { return preferredSpillBatchSize; }
   @VisibleForTesting
   public int getPreferredMergeBatchSize() { return preferredMergeBatchSize; }
   @VisibleForTesting
-  public int getSpillBatchSize() { return expectedSpillBatchSize; }
+  public BatchSizeEstimate getSpillBatchSize() { return spillBatchSize; }
   @VisibleForTesting
-  public int getMergeBatchSize() { return expectedMergeBatchSize; }
+  public BatchSizeEstimate getMergeBatchSize() { return mergeBatchSize; }
   @VisibleForTesting
   public long getBufferMemoryLimit() { return bufferMemoryLimit; }
   @VisibleForTesting
   public boolean mayOverflow() { return potentialOverflow; }
+  @VisibleForTesting
+  public boolean isLowMemory() { return isLowMemory; }
+  @VisibleForTesting
+  public boolean hasPerformanceWarning() { return performanceWarning; }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
index 4231cf4..0f27884 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
@@ -83,10 +83,9 @@ public class SorterWrapper extends BaseSortWrapper {
     ClassGenerator<SingleBatchSorter> g = cg.getRoot();
     cg.plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.
-  cg.saveCodeForDebugging(true);
+//    cg.saveCodeForDebugging(true);
 
     generateComparisons(g, batch, logger);
     return getInstance(cg, logger);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
index a6042c6..b75ce77 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.physical.impl.spill.SpillSet;
 import org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.SpilledRun;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorInitializer;
 import org.apache.drill.exec.record.VectorContainer;
 
 import com.google.common.collect.Lists;
@@ -86,13 +87,14 @@ public class SpilledRuns {
     return batchesToSpill;
   }
 
-  public void mergeAndSpill(List<BatchGroup> batchesToSpill, int spillBatchRowCount) {
-    spilledRuns.add(safeMergeAndSpill(batchesToSpill, spillBatchRowCount));
+  public void mergeAndSpill(List<BatchGroup> batchesToSpill, int spillBatchRowCount, VectorInitializer allocHelper) {
+    spilledRuns.add(safeMergeAndSpill(batchesToSpill, spillBatchRowCount, allocHelper));
     logger.trace("Completed spill: memory = {}",
         context.getAllocator().getAllocatedMemory());
   }
 
-  public void mergeRuns(int targetCount, long mergeMemoryPool, int spillBatchRowCount) {
+  public void mergeRuns(int targetCount, long mergeMemoryPool,
+                  int spillBatchRowCount, VectorInitializer allocHelper) {
 
     long allocated = context.getAllocator().getAllocatedMemory();
     mergeMemoryPool -= context.getAllocator().getAllocatedMemory();
@@ -128,12 +130,12 @@ public class SpilledRuns {
     // Do the actual spill.
 
     List<BatchGroup> batchesToSpill = prepareSpillBatches(spilledRuns, mergeCount);
-    mergeAndSpill(batchesToSpill, spillBatchRowCount);
+    mergeAndSpill(batchesToSpill, spillBatchRowCount, allocHelper);
   }
 
-  private BatchGroup.SpilledRun safeMergeAndSpill(List<? extends BatchGroup> batchesToSpill, int spillBatchRowCount) {
+  private BatchGroup.SpilledRun safeMergeAndSpill(List<? extends BatchGroup> batchesToSpill, int spillBatchRowCount, VectorInitializer allocHelper) {
     try {
-      return doMergeAndSpill(batchesToSpill, spillBatchRowCount);
+      return doMergeAndSpill(batchesToSpill, spillBatchRowCount, allocHelper);
     }
     // If error is a User Exception, just use as is.
 
@@ -145,7 +147,8 @@ public class SpilledRuns {
     }
   }
 
-  private BatchGroup.SpilledRun doMergeAndSpill(List<? extends BatchGroup> batchesToSpill, int spillBatchRowCount) throws Throwable {
+  private BatchGroup.SpilledRun doMergeAndSpill(List<? extends BatchGroup> batchesToSpill,
+                        int spillBatchRowCount, VectorInitializer allocHelper) throws Throwable {
 
     // Merge the selected set of matches and write them to the
     // spill file. After each write, we release the memory associated
@@ -155,7 +158,8 @@ public class SpilledRuns {
     BatchGroup.SpilledRun newGroup = null;
     VectorContainer dest = new VectorContainer();
     try (AutoCloseable ignored = AutoCloseables.all(batchesToSpill);
-         PriorityQueueCopierWrapper.BatchMerger merger = copierHolder.startMerge(schema, batchesToSpill, dest, spillBatchRowCount)) {
+         PriorityQueueCopierWrapper.BatchMerger merger = copierHolder.startMerge(schema, batchesToSpill,
+                                         dest, spillBatchRowCount, allocHelper)) {
       newGroup = new BatchGroup.SpilledRun(spillSet, outputFile, context.getAllocator());
       logger.trace("Spilling {} batches, into spill batches of {} rows, to {}",
           batchesToSpill.size(), spillBatchRowCount, outputFile);
@@ -175,9 +179,9 @@ public class SpilledRuns {
       }
       context.injectChecked(ExternalSortBatch.INTERRUPTION_WHILE_SPILLING, IOException.class);
       newGroup.closeOutputStream();
-      logger.trace("Spilled {} output batches, each of {} by bytes, {} records to {}",
-                   merger.getBatchCount(), merger.getRecordCount(),
-                   merger.getEstBatchSize(), outputFile);
+      logger.trace("Spilled {} output batches, each of {} bytes, {} records, to {}",
+                   merger.getBatchCount(), merger.getEstBatchSize(),
+                   spillBatchRowCount, outputFile);
       newGroup.setBatchSize(merger.getEstBatchSize());
       return newGroup;
     } catch (Throwable e) {
@@ -192,7 +196,8 @@ public class SpilledRuns {
     }
   }
 
-  public SortResults finalMerge(List<? extends BatchGroup> bufferedBatches, VectorContainer container, int mergeRowCount) {
+  public SortResults finalMerge(List<? extends BatchGroup> bufferedBatches,
+                    VectorContainer container, int mergeRowCount, VectorInitializer allocHelper) {
     List<BatchGroup> allBatches = new LinkedList<>();
     allBatches.addAll(bufferedBatches);
     bufferedBatches.clear();
@@ -200,7 +205,7 @@ public class SpilledRuns {
     spilledRuns.clear();
     logger.debug("Starting merge phase. Runs = {}, Alloc. memory = {}",
         allBatches.size(), context.getAllocator().getAllocatedMemory());
-    return copierHolder.startMerge(schema, allBatches, container, mergeRowCount);
+    return copierHolder.startMerge(schema, allBatches, container, mergeRowCount, allocHelper);
   }
 
   public void close() {

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index ca275c7..eb90614 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -117,17 +117,19 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
         return IterOutcome.STOP;
       }
       next = b.next();
-    }finally{
+    } finally {
       stats.startProcessing();
     }
 
-    switch(next){
+    switch(next) {
     case OK_NEW_SCHEMA:
       stats.batchReceived(inputIndex, b.getRecordCount(), true);
       break;
     case OK:
       stats.batchReceived(inputIndex, b.getRecordCount(), false);
       break;
+    default:
+      break;
     }
 
     return next;

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
index 44c6b1a..4e47051 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/HyperVectorWrapper.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -81,6 +81,7 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
     }
   }
 
+  @SuppressWarnings("resource")
   @Override
   public VectorWrapper<?> getChildWrapper(int[] ids) {
     if (ids.length == 1) {
@@ -105,6 +106,7 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
     return new HyperVectorWrapper<ValueVector>(vectors[0].getField(), vectors);
   }
 
+  @SuppressWarnings("resource")
   @Override
   public TypedFieldId getFieldIdIfMatches(int id, SchemaPath expectedPath) {
     ValueVector v = vectors[0];
@@ -112,7 +114,6 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public VectorWrapper<T> cloneAndTransfer(BufferAllocator allocator) {
     return new HyperVectorWrapper<T>(f, vectors, false);
 //    T[] newVectors = (T[]) Array.newInstance(vectors.getClass().getComponentType(), vectors.length);
@@ -128,12 +129,14 @@ public class HyperVectorWrapper<T extends ValueVector> implements VectorWrapper<
     return new HyperVectorWrapper<T>(f, v, releasable);
   }
 
+  @SuppressWarnings("unchecked")
   public void addVector(ValueVector v) {
     Preconditions.checkArgument(v.getClass() == this.getVectorClass(), String.format("Cannot add vector type %s to hypervector type %s for field %s",
       v.getClass(), this.getVectorClass(), v.getField()));
     vectors = (T[]) ArrayUtils.add(vectors, v);// TODO optimize this so not copying every time
   }
 
+  @SuppressWarnings("unchecked")
   public void addVectors(ValueVector[] vv) {
     vectors = (T[]) ArrayUtils.add(vectors, vv);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 0daa6b3..b4ae2d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,8 +19,6 @@ package org.apache.drill.exec.record;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.record.selection.SelectionVector4;
 
 /**
  * A record batch contains a set of field values for a particular range of

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorInitializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorInitializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorInitializer.java
new file mode 100644
index 0000000..5dd348e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorInitializer.java
@@ -0,0 +1,154 @@
+/*
+ * 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.drill.exec.record;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+
+/**
+ * Prototype mechanism to allocate vectors based on expected
+ * data sizes. This version uses a name-based map of fields
+ * to sizes. Better to represent the batch structurally and
+ * simply iterate over the schema rather than doing a per-field
+ * lookup. But, the mechanisms needed to do the efficient solution
+ * don't exist yet.
+ */
+
+public class VectorInitializer {
+
+  private static class AllocationHint {
+    public final int entryWidth;
+    public final int elementCount;
+
+    private AllocationHint(int width, int elements) {
+      entryWidth = width;
+      elementCount = elements;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder()
+          .append("{");
+      String sep = "";
+      if (entryWidth > 0) {
+        buf.append("width=")
+           .append(entryWidth);
+        sep = ", ";
+      }
+      if (elementCount > 0) {
+        buf.append(sep)
+           .append("elements=")
+           .append(elementCount);
+      }
+      buf.append("}");
+      return buf.toString();
+    }
+  }
+
+  private Map<String, AllocationHint> hints = new HashMap<>();
+
+  public void variableWidth(String name, int width) {
+    hints.put(name, new AllocationHint(width, 1));
+  }
+
+  public void fixedWidthArray(String name, int elements) {
+    hints.put(name, new AllocationHint(0, elements));
+  }
+
+  public void variableWidthArray(String name, int width, int elements) {
+    hints.put(name, new AllocationHint(width, elements));
+  }
+
+  public void allocateBatch(VectorAccessible va, int recordCount) {
+    for (VectorWrapper<?> w: va) {
+      allocateVector(w.getValueVector(), "", recordCount);
+    }
+  }
+
+  private void allocateVector(ValueVector vector, String prefix, int recordCount) {
+    String key = prefix + vector.getField().getName();
+    AllocationHint hint = hints.get(key);
+    if (vector instanceof AbstractMapVector) {
+      allocateMap((AbstractMapVector) vector, prefix, recordCount, hint);
+    } else {
+      allocateVector(vector, recordCount, hint);
+    }
+//    Set<BufferLedger> ledgers = new HashSet<>();
+//    vector.getLedgers(ledgers);
+//    int size = 0;
+//    for (BufferLedger ledger : ledgers) {
+//      size += ledger.getAccountedSize();
+//    }
+//    System.out.println(key + ": " + vector.getField().toString() +
+//        " " +
+//        ((hint == null) ? "no hint" : hint.toString()) +
+//        ", " + size);
+  }
+
+  private void allocateVector(ValueVector vector, int recordCount, AllocationHint hint) {
+    if (hint == null) {
+      // Use hard-coded values. Same as ScanBatch
+
+      AllocationHelper.allocate(vector, recordCount, 50, 10);
+    } else {
+      AllocationHelper.allocate(vector, recordCount, hint.entryWidth, hint.elementCount);
+    }
+  }
+
+  private void allocateMap(AbstractMapVector map, String prefix, int recordCount, AllocationHint hint) {
+    if (map instanceof RepeatedMapVector) {
+      ((RepeatedMapVector) map).allocateOffsetsNew(recordCount);
+      if (hint == null) {
+        recordCount *= 10;
+      } else {
+        recordCount *= hint.elementCount;
+      }
+    }
+    prefix += map.getField().getName() + ".";
+    for (ValueVector vector : map) {
+      allocateVector(vector, prefix, recordCount);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("[" + getClass().getSimpleName())
+       .append(" ");
+    boolean first = true;
+    for (Entry<String, AllocationHint>entry : hints.entrySet()) {
+      if (! first) {
+        buf.append(", ");
+      }
+      first = false;
+      buf.append("[")
+         .append(entry.getKey())
+         .append(" ")
+         .append(entry.getValue().toString())
+         .append("]");
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index bcec920..b3b46c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -80,6 +80,7 @@ public class WritableBatch implements AutoCloseable {
         len += b.capacity();
       }
 
+      @SuppressWarnings("resource")
       DrillBuf newBuf = allocator.buffer(len);
       try {
         /* Copy data from each buffer into the compound buffer */
@@ -101,7 +102,9 @@ public class WritableBatch implements AutoCloseable {
 
         for (VectorWrapper<?> vv : container) {
           SerializedField fmd = fields.get(vectorIndex);
+          @SuppressWarnings("resource")
           ValueVector v = vv.getValueVector();
+          @SuppressWarnings("resource")
           DrillBuf bb = newBuf.slice(bufferOffset, fmd.getBufferLength());
 //        v.load(fmd, cbb.slice(bufferOffset, fmd.getBufferLength()));
           v.load(fmd, bb);

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 0d341df..7ed9220 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -293,6 +293,7 @@ public class Drillbit implements AutoCloseable {
     return start(config, null);
   }
 
+  @SuppressWarnings("resource")
   public static Drillbit start(final DrillConfig config, final RemoteServiceSet remoteServiceSet)
       throws DrillbitStartupException {
     logger.debug("Starting new Drillbit.");

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 437862e..41ecc95 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -251,8 +251,6 @@ drill.exec: {
     external: {
       // Drill uses the managed External Sort Batch by default.
       // Set this to true to use the legacy, unmanaged version.
-      // Disabled in the intial commit, to be enabled after
-      // tests are committed.
       disable_managed: true,
       // Limit on the number of batches buffered in memory.
       // Primarily for testing.
@@ -282,9 +280,9 @@ drill.exec: {
         directories:  ${drill.exec.spill.directories},
         // Size of the batches written to, and read from, the spill files.
         // Determines the ratio of memory to input data size for a single-
-        // generation sort. Smaller values give larger ratios, but at a
-        // (high) cost of much greater disk seek times.
-        spill_batch_size = 8M,
+        // generation sort. Smaller values are better, but too small
+        // incurs per-batch overhead.
+        spill_batch_size = 1M,
         // Preferred file size for "first-generation" spill files.
         // Set large enough to get long, continuous writes, but not so
         // large as to overwhelm a temp directory.
@@ -292,7 +290,8 @@ drill.exec: {
         file_size: 256M,
         // Size of the batch sent downstream from the sort operator during
         // the merge phase. Don't change this unless you know what you are doing,
-        // larger sizes can result in memory fragmentation.
+        // larger sizes can result in memory fragmentation, smaller sizes
+        // in excessive operator iterator overhead.
         merge_batch_size = 16M
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
index 7700a1e..ee350ce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
@@ -31,7 +31,6 @@ import org.junit.Test;
 import java.util.List;
 
 public class TestUnionAll extends BaseTestQuery{
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestUnionAll.class);
 
   private static final String sliceTargetSmall = "alter session set `planner.slice_target` = 1";
   private static final String sliceTargetDefault = "alter session reset `planner.slice_target`";

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
index 05670c5..cfb8645 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.cache;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.File;
@@ -119,8 +116,6 @@ public class TestBatchSerialization extends DrillTest {
    */
   private void verifySerialize(SingleRowSet rowSet, SingleRowSet expected) throws IOException {
 
-    long origSize = rowSet.size();
-
     File dir = OperatorFixture.getTempDir("serial");
     File outFile = new File(dir, "serialze.dat");
     try (OutputStream out = new BufferedOutputStream(new FileOutputStream(outFile))) {
@@ -135,7 +130,6 @@ public class TestBatchSerialization extends DrillTest {
           .read());
     }
 
-    assertTrue(origSize >= result.size());
     new RowSetComparison(expected)
       .verifyAndClearAll(result);
     outFile.delete();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
index 76f0935..73f9b6d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
@@ -440,6 +440,11 @@ public class TestWindowFrame extends BaseTestQuery {
       .go();
   }
 
+  // Note: This test is unstable. It works when forcing the merge/sort batch
+  // size to 20, but not for other sizes. The problem is either that the results
+  // are not ordered (and so subject to sort instability), or there is some bug
+  // somewhere in the window functions.
+
   @Test
   public void test4657() throws Exception {
     testBuilder()

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index f643d5f..7e63600 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -35,14 +35,17 @@ import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.FixtureBuilder;
+import org.apache.drill.test.SecondaryTest;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
+@Category(SecondaryTest.class)
 public class TestSimpleExternalSort extends DrillTest {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleExternalSort.class);
 
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(80000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(80_000);
 
   @Test
   public void mergeSortWithSv2Managed() throws Exception {
@@ -100,7 +103,7 @@ public class TestSimpleExternalSort extends DrillTest {
          ClientFixture client = cluster.clientFixture()) {
       chooseImpl(client, testLegacy);
       List<QueryDataBatch> results = client.queryBuilder().physicalResource("xsort/one_key_sort_descending.json").results();
-      assertEquals(1000000, client.countResults(results));
+      assertEquals(1_000_000, client.countResults(results));
       validateResults(client.allocator(), results);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
index 5a1bf6d..bbb48af 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
@@ -64,8 +64,8 @@ public class TestSortSpillWithException extends ClusterTest {
     // inject exception in sort while spilling
     final String controls = Controls.newBuilder()
       .addExceptionOnBit(
-          org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.class,
-          org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
+          ExternalSortBatch.class,
+          ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
           IOException.class,
           cluster.drillbit().getContext().getEndpoint())
       .build();
@@ -87,8 +87,8 @@ public class TestSortSpillWithException extends ClusterTest {
     // inject exception in sort while spilling
     final String controls = Controls.newBuilder()
       .addExceptionOnBit(
-          org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.class,
-          org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
+          ExternalSortBatch.class,
+          ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
           IOException.class,
           cluster.drillbit().getContext().getEndpoint())
       .build();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
index 1a4d4b2..8ba34ef 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -104,7 +104,7 @@ public class SortTestUtilities {
       VectorContainer dest = new VectorContainer();
       @SuppressWarnings("resource")
       BatchMerger merger = copier.startMerge(schema.toBatchSchema(SelectionVectorMode.NONE),
-                                             batches, dest, rowCount);
+                                             batches, dest, rowCount, null);
 
       verifyResults(merger, dest);
       dest.clear();

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
index 0050747..6464b5a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
@@ -69,8 +69,13 @@ public class TestCopier extends DrillTest {
     PriorityQueueCopierWrapper copier = SortTestUtilities.makeCopier(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED);
     VectorContainer dest = new VectorContainer();
     try {
+      // TODO: Create a vector allocator to pass as last parameter so
+      // that the test uses the same vector allocator as the production
+      // code. Only nuisance is that we don't have the required metadata
+      // readily at hand here...
+
       @SuppressWarnings({ "resource", "unused" })
-      BatchMerger merger = copier.startMerge(schema, batches, dest, 10);
+      BatchMerger merger = copier.startMerge(schema, batches, dest, 10, null);
       fail();
     } catch (AssertionError e) {
       // Expected

http://git-wip-us.apache.org/repos/asf/drill/blob/073ea681/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
index 6bff088..69e9e1b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
@@ -47,12 +47,14 @@ public class TestExternalSortInternals extends DrillTest {
     assertEquals(Integer.MAX_VALUE, sortConfig.mergeLimit());
     // Default size: 256 MiB
     assertEquals(256 * ONE_MEG, sortConfig.spillFileSize());
-    // Default size: 8 MiB
-    assertEquals(8 * ONE_MEG, sortConfig.spillBatchSize());
+    // Default size: 1 MiB
+    assertEquals(ONE_MEG, sortConfig.spillBatchSize());
     // Default size: 16 MiB
     assertEquals(16 * ONE_MEG, sortConfig.mergeBatchSize());
     // Default: unlimited
     assertEquals(Integer.MAX_VALUE, sortConfig.getBufferedBatchLimit());
+    // Default: 64K
+    assertEquals(Character.MAX_VALUE, sortConfig.getMSortBatchSize());
   }
 
   /**
@@ -69,6 +71,7 @@ public class TestExternalSortInternals extends DrillTest {
         .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, 500_000)
         .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, 600_000)
         .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 50)
+        .put(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE, 10)
         .build();
     SortConfig sortConfig = new SortConfig(drillConfig);
     assertEquals(2000 * 1024, sortConfig.maxMemory());
@@ -77,6 +80,7 @@ public class TestExternalSortInternals extends DrillTest {
     assertEquals(500_000, sortConfig.spillBatchSize());
     assertEquals(600_000, sortConfig.mergeBatchSize());
     assertEquals(50, sortConfig.getBufferedBatchLimit());
+    assertEquals(10, sortConfig.getMSortBatchSize());
   }
 
   /**
@@ -90,6 +94,7 @@ public class TestExternalSortInternals extends DrillTest {
         .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, SortConfig.MIN_SPILL_BATCH_SIZE - 1)
         .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, SortConfig.MIN_MERGE_BATCH_SIZE - 1)
         .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 1)
+        .put(ExecConstants.EXTERNAL_SORT_MSORT_MAX_BATCHSIZE, 0)
         .build();
     SortConfig sortConfig = new SortConfig(drillConfig);
     assertEquals(SortConfig.MIN_MERGE_LIMIT, sortConfig.mergeLimit());
@@ -97,13 +102,14 @@ public class TestExternalSortInternals extends DrillTest {
     assertEquals(SortConfig.MIN_SPILL_BATCH_SIZE, sortConfig.spillBatchSize());
     assertEquals(SortConfig.MIN_MERGE_BATCH_SIZE, sortConfig.mergeBatchSize());
     assertEquals(2, sortConfig.getBufferedBatchLimit());
+    assertEquals(1, sortConfig.getMSortBatchSize());
   }
 
   @Test
   public void testMemoryManagerBasics() {
     DrillConfig drillConfig = DrillConfig.create();
     SortConfig sortConfig = new SortConfig(drillConfig);
-    long memoryLimit = 50 * ONE_MEG;
+    long memoryLimit = 70 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
     // Basic setup
@@ -120,35 +126,35 @@ public class TestExternalSortInternals extends DrillTest {
     int rowCount = 10000;
     int batchSize = rowWidth * rowCount * 2;
 
-    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertTrue(memManager.updateEstimates(batchSize, rowWidth, rowCount));
     verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
 
     // Zero rows - no update
 
-    memManager.updateEstimates(batchSize, rowWidth, 0);
+    assertFalse(memManager.updateEstimates(batchSize, rowWidth, 0));
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Larger batch size, update batch size
 
     rowCount = 20000;
     batchSize = rowWidth * rowCount * 2;
-    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertTrue(memManager.updateEstimates(batchSize, rowWidth, rowCount));
     verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
 
     // Smaller batch size: no change
 
     rowCount = 5000;
     int lowBatchSize = rowWidth * rowCount * 2;
-    memManager.updateEstimates(lowBatchSize, rowWidth, rowCount);
+    assertFalse(memManager.updateEstimates(lowBatchSize, rowWidth, rowCount));
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Different batch density, update batch size
 
     rowCount = 10000;
     batchSize = rowWidth * rowCount * 5;
-    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertTrue(memManager.updateEstimates(batchSize, rowWidth, rowCount));
     verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
 
     // Smaller row size, no update
@@ -156,23 +162,23 @@ public class TestExternalSortInternals extends DrillTest {
     int lowRowWidth = 200;
     rowCount = 10000;
     lowBatchSize = rowWidth * rowCount * 2;
-    memManager.updateEstimates(lowBatchSize, lowRowWidth, rowCount);
+    assertFalse(memManager.updateEstimates(lowBatchSize, lowRowWidth, rowCount));
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Larger row size, updates calcs
 
     rowWidth = 400;
     rowCount = 10000;
     lowBatchSize = rowWidth * rowCount * 2;
-    memManager.updateEstimates(lowBatchSize, rowWidth, rowCount);
+    assertTrue(memManager.updateEstimates(lowBatchSize, rowWidth, rowCount));
     verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
 
     // EOF: very low density
 
-    memManager.updateEstimates(lowBatchSize, rowWidth, 5);
+    assertFalse(memManager.updateEstimates(lowBatchSize, rowWidth, 5));
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
   }
 
   private void verifyCalcs(SortConfig sortConfig, long memoryLimit, SortMemoryManager memManager, int batchSize,
@@ -183,7 +189,7 @@ public class TestExternalSortInternals extends DrillTest {
     // Row and batch sizes should be exact
 
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Spill sizes will be rounded, but within reason.
 
@@ -191,9 +197,9 @@ public class TestExternalSortInternals extends DrillTest {
     assertTrue(count >= memManager.getSpillBatchRowCount());
     assertTrue(count/2 <= memManager.getSpillBatchRowCount());
     int spillSize = memManager.getSpillBatchRowCount() * rowWidth;
-    assertTrue(spillSize <= memManager.getSpillBatchSize());
-    assertTrue(spillSize >= memManager.getSpillBatchSize()/2);
-    assertEquals(memoryLimit - memManager.getSpillBatchSize(), memManager.getBufferMemoryLimit());
+    assertTrue(spillSize <= memManager.getSpillBatchSize().dataSize);
+    assertTrue(spillSize >= memManager.getSpillBatchSize().dataSize/2);
+    assertTrue(memManager.getBufferMemoryLimit() <= memoryLimit - memManager.getSpillBatchSize().expectedBufferSize );
 
     // Merge sizes will also be rounded, within reason.
 
@@ -201,9 +207,9 @@ public class TestExternalSortInternals extends DrillTest {
     assertTrue(count >= memManager.getMergeBatchRowCount());
     assertTrue(count/2 <= memManager.getMergeBatchRowCount());
     int mergeSize = memManager.getMergeBatchRowCount() * rowWidth;
-    assertTrue(mergeSize <= memManager.getMergeBatchSize());
-    assertTrue(mergeSize >= memManager.getMergeBatchSize()/2);
-    assertEquals(memoryLimit - memManager.getMergeBatchSize(), memManager.getMergeMemoryLimit());
+    assertTrue(mergeSize <= memManager.getMergeBatchSize().dataSize);
+    assertTrue(mergeSize >= memManager.getMergeBatchSize().dataSize/2);
+    assertTrue(memManager.getMergeMemoryLimit() <= memoryLimit - memManager.getMergeBatchSize().expectedBufferSize);
   }
 
   @Test
@@ -220,7 +226,7 @@ public class TestExternalSortInternals extends DrillTest {
     int batchSize = rowCount * 2;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertEquals(10, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Truncate spill, merge batch row count
 
@@ -234,12 +240,12 @@ public class TestExternalSortInternals extends DrillTest {
 
     // Small, but non-zero, row
 
-    rowWidth = 20;
+    rowWidth = 10;
     rowCount = 10000;
     batchSize = rowWidth * rowCount;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
 
     // Truncate spill, merge batch row count
 
@@ -256,69 +262,89 @@ public class TestExternalSortInternals extends DrillTest {
   public void testLowMemory() {
     DrillConfig drillConfig = DrillConfig.create();
     SortConfig sortConfig = new SortConfig(drillConfig);
-    long memoryLimit = 10 * ONE_MEG;
+    int memoryLimit = 10 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
     // Tight squeeze, but can be made to work.
-    // Input batches are a quarter of memory.
+    // Input batch buffer size is a quarter of memory.
 
     int rowWidth = 1000;
-    int rowCount = (int) (memoryLimit / 4 / rowWidth);
-    int batchSize = rowCount * rowWidth;
+    int batchSize = SortMemoryManager.multiply(memoryLimit / 4, SortMemoryManager.PAYLOAD_FROM_BUFFER);
+    int rowCount = batchSize / rowWidth;
+    batchSize = rowCount * rowWidth;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
     assertFalse(memManager.mayOverflow());
+    assertTrue(memManager.hasPerformanceWarning());
 
     // Spill, merge batches should be constrained
 
-    int spillBatchSize = memManager.getSpillBatchSize();
+    int spillBatchSize = memManager.getSpillBatchSize().dataSize;
     assertTrue(spillBatchSize < memManager.getPreferredSpillBatchSize());
     assertTrue(spillBatchSize >= rowWidth);
     assertTrue(spillBatchSize <= memoryLimit / 3);
     assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
     assertTrue(spillBatchSize / rowWidth >= memManager.getSpillBatchRowCount());
 
-    int mergeBatchSize = memManager.getMergeBatchSize();
+    int mergeBatchSize = memManager.getMergeBatchSize().dataSize;
     assertTrue(mergeBatchSize < memManager.getPreferredMergeBatchSize());
     assertTrue(mergeBatchSize >= rowWidth);
     assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
     assertTrue(mergeBatchSize / rowWidth >= memManager.getMergeBatchRowCount());
 
-    // Should spill after just two batches
+    // Should spill after just two or three batches
 
-    assertFalse(memManager.isSpillNeeded(0, batchSize));
-    assertFalse(memManager.isSpillNeeded(batchSize, batchSize));
-    assertTrue(memManager.isSpillNeeded(2 * batchSize, batchSize));
+    int inputBufferSize = memManager.getInputBatchSize().expectedBufferSize;
+    assertFalse(memManager.isSpillNeeded(0, inputBufferSize));
+    assertFalse(memManager.isSpillNeeded(batchSize, inputBufferSize));
+    assertTrue(memManager.isSpillNeeded(3 * inputBufferSize, inputBufferSize));
+  }
+
+  @Test
+  public void testLowerMemory() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    int memoryLimit = 10 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
     // Tighter squeeze, but can be made to work.
     // Input batches are 3/8 of memory; two fill 3/4,
     // but small spill and merge batches allow progress.
 
-    rowWidth = 1000;
-    rowCount = (int) (memoryLimit * 3 / 8 / rowWidth);
+    int rowWidth = 1000;
+    int batchSize = SortMemoryManager.multiply(memoryLimit * 3 / 8, SortMemoryManager.PAYLOAD_FROM_BUFFER);
+    int rowCount = batchSize / rowWidth;
     batchSize = rowCount * rowWidth;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
     assertFalse(memManager.mayOverflow());
+    assertTrue(memManager.hasPerformanceWarning());
 
     // Spill, merge batches should be constrained
 
-    spillBatchSize = memManager.getSpillBatchSize();
+    int spillBatchSize = memManager.getSpillBatchSize().dataSize;
     assertTrue(spillBatchSize < memManager.getPreferredSpillBatchSize());
     assertTrue(spillBatchSize >= rowWidth);
     assertTrue(spillBatchSize <= memoryLimit / 3);
     assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
-    assertTrue(memManager.getSpillBatchRowCount() > 1);
+    assertTrue(memManager.getSpillBatchRowCount() >= 1);
     assertTrue(spillBatchSize / rowWidth >= memManager.getSpillBatchRowCount());
 
-    mergeBatchSize = memManager.getMergeBatchSize();
+    int mergeBatchSize = memManager.getMergeBatchSize().dataSize;
     assertTrue(mergeBatchSize < memManager.getPreferredMergeBatchSize());
     assertTrue(mergeBatchSize >= rowWidth);
     assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
     assertTrue(memManager.getMergeBatchRowCount() > 1);
     assertTrue(mergeBatchSize / rowWidth >= memManager.getMergeBatchRowCount());
+
+    // Should spill after just two batches
+
+    int inputBufferSize = memManager.getInputBatchSize().expectedBufferSize;
+    assertFalse(memManager.isSpillNeeded(0, inputBufferSize));
+    assertFalse(memManager.isSpillNeeded(batchSize, inputBufferSize));
+    assertTrue(memManager.isSpillNeeded(2 * inputBufferSize, inputBufferSize));
   }
 
   @Test
@@ -333,21 +359,22 @@ public class TestExternalSortInternals extends DrillTest {
     // Have to back off the exact size a bit to allow for internal fragmentation
     // in the merge and output batches.
 
-    int rowWidth = (int) (memoryLimit / 3 * 0.75);
+    int rowWidth = (int) (memoryLimit / 3 / 2);
     int rowCount = 1;
     int batchSize = rowWidth;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertEquals(rowWidth, memManager.getRowWidth());
-    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertEquals(batchSize, memManager.getInputBatchSize().dataSize);
     assertFalse(memManager.mayOverflow());
+    assertTrue(memManager.hasPerformanceWarning());
 
-    int spillBatchSize = memManager.getSpillBatchSize();
+    int spillBatchSize = memManager.getSpillBatchSize().dataSize;
     assertTrue(spillBatchSize >= rowWidth);
     assertTrue(spillBatchSize <= memoryLimit / 3);
     assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
     assertEquals(1, memManager.getSpillBatchRowCount());
 
-    int mergeBatchSize = memManager.getMergeBatchSize();
+    int mergeBatchSize = memManager.getMergeBatchSize().dataSize;
     assertTrue(mergeBatchSize >= rowWidth);
     assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
     assertEquals(1, memManager.getMergeBatchRowCount());
@@ -357,12 +384,26 @@ public class TestExternalSortInternals extends DrillTest {
     assertFalse(memManager.isSpillNeeded(0, batchSize));
     assertFalse(memManager.isSpillNeeded(batchSize, batchSize));
     assertTrue(memManager.isSpillNeeded(2 * batchSize, batchSize));
+  }
 
-    // In trouble now, can't fit even three rows.
+  @Test
+  public void testMemoryOverflow() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 10 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // In trouble now, can't fit even two input batches.
+    // A better implementation would spill the first batch to a file,
+    // leave it open, and append the second batch. Slicing each big input
+    // batch into small spill batches will allow the sort to proceed as
+    // long as it can hold a single input batch and single merge batch. But,
+    // the current implementation requires all batches to be spilled are in
+    // memory at the same time...
 
-    rowWidth = (int) (memoryLimit / 2);
-    rowCount = 1;
-    batchSize = rowWidth;
+    int rowWidth = (int) (memoryLimit / 2);
+    int rowCount = 1;
+    int batchSize = rowWidth;
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
     assertTrue(memManager.mayOverflow());
   }
@@ -406,7 +447,7 @@ public class TestExternalSortInternals extends DrillTest {
 
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
 
-    int spillBatchSize = memManager.getSpillBatchSize();
+    int spillBatchSize = memManager.getSpillBatchSize().dataSize;
 
     // Test various memory fill levels
 
@@ -432,63 +473,67 @@ public class TestExternalSortInternals extends DrillTest {
         .build();
     SortConfig sortConfig = new SortConfig(drillConfig);
     // Allow four spill batches, 8 MB each, plus one output of 16
-    long memoryLimit = 50 * ONE_MEG;
+    // Allow for internal fragmentation
+    // 96 > (4 * 8 + 16) * 2
+    long memoryLimit = 96 * ONE_MEG;
     SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
 
-    // Prime the estimates
+    // Prime the estimates. Batch size is data size, not buffer size.
 
     int rowWidth = 300;
     int rowCount = 10000;
     int batchSize = rowWidth * rowCount * 2;
 
     memManager.updateEstimates(batchSize, rowWidth, rowCount);
-    int spillBatchSize = memManager.getSpillBatchSize();
-    int mergeBatchSize = memManager.getMergeBatchSize();
+    assertFalse(memManager.isLowMemory());
+    int spillBatchBufferSize = memManager.getSpillBatchSize().expectedBufferSize;
+    int inputBatchBufferSize = memManager.getInputBatchSize().expectedBufferSize;
 
     // One in-mem batch, no merging.
 
-    long allocMemory = memoryLimit - mergeBatchSize;
+    long allocMemory = inputBatchBufferSize;
     MergeTask task = memManager.consolidateBatches(allocMemory, 1, 0);
     assertEquals(MergeAction.NONE, task.action);
 
     // Many in-mem batches, just enough to merge
 
-    allocMemory = memoryLimit - mergeBatchSize;
-    int memBatches = (int) (allocMemory / batchSize);
-    allocMemory = memBatches * batchSize;
+    int memBatches = (int) (memManager.getMergeMemoryLimit() / inputBatchBufferSize);
+    allocMemory = memBatches * inputBatchBufferSize;
     task = memManager.consolidateBatches(allocMemory, memBatches, 0);
     assertEquals(MergeAction.NONE, task.action);
 
     // Spills if no room for spill and in-memory batches
 
-    task = memManager.consolidateBatches(allocMemory, memBatches, 1);
+    int spillCount = (int) Math.ceil((memManager.getMergeMemoryLimit() - allocMemory) / (1.0 * spillBatchBufferSize));
+    assertTrue(spillCount >= 1);
+    task = memManager.consolidateBatches(allocMemory, memBatches, spillCount);
     assertEquals(MergeAction.SPILL, task.action);
 
     // One more in-mem batch: now needs to spill
 
     memBatches++;
-    allocMemory = memBatches * batchSize;
+    allocMemory = memBatches * inputBatchBufferSize;
     task = memManager.consolidateBatches(allocMemory, memBatches, 0);
     assertEquals(MergeAction.SPILL, task.action);
 
     // No spill for various in-mem/spill run combinations
 
-    allocMemory = memoryLimit - spillBatchSize - mergeBatchSize;
-    memBatches = (int) (allocMemory / batchSize);
-    allocMemory = memBatches * batchSize;
+    long freeMem = memManager.getMergeMemoryLimit() - spillBatchBufferSize;
+    memBatches = (int) (freeMem / inputBatchBufferSize);
+    allocMemory = memBatches * inputBatchBufferSize;
     task = memManager.consolidateBatches(allocMemory, memBatches, 1);
     assertEquals(MergeAction.NONE, task.action);
 
-    allocMemory = memoryLimit - 2 * spillBatchSize - mergeBatchSize;
-    memBatches = (int) (allocMemory / batchSize);
-    allocMemory = memBatches * batchSize;
+    freeMem = memManager.getMergeMemoryLimit() - 2 * spillBatchBufferSize;
+    memBatches = (int) (freeMem / inputBatchBufferSize);
+    allocMemory = memBatches * inputBatchBufferSize;
     task = memManager.consolidateBatches(allocMemory, memBatches, 2);
     assertEquals(MergeAction.NONE, task.action);
 
     // No spill if no in-memory, only spill, and spill fits
 
-    long freeMem = memoryLimit - mergeBatchSize;
-    int spillBatches = (int) (freeMem / spillBatchSize);
+    freeMem = memManager.getMergeMemoryLimit();
+    int spillBatches = (int) (freeMem / spillBatchBufferSize);
     task = memManager.consolidateBatches(0, 0, spillBatches);
     assertEquals(MergeAction.NONE, task.action);
 
@@ -503,6 +548,47 @@ public class TestExternalSortInternals extends DrillTest {
     task = memManager.consolidateBatches(0, 0, spillBatches + 2);
     assertEquals(MergeAction.MERGE, task.action);
     assertEquals(3, task.count);
+
+    // If only one spilled run, and no in-memory batches,
+    // skip merge.
+
+    task = memManager.consolidateBatches(0, 0, 1);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // Very large number of spilled runs. Limit to what fits in memory.
+
+    task = memManager.consolidateBatches(0, 0, 1000);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertTrue(task.count <= (int)(memoryLimit / spillBatchBufferSize) - 1);
+  }
+
+  @Test
+  public void testMergeCalcsExtreme() {
+
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+
+    // Force odd situation in which the spill batch is larger
+    // than memory. Won't actually run, but needed to test
+    // odd merge case.
+
+    long memoryLimit = ONE_MEG / 2;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Prime the estimates. Batch size is data size, not buffer size.
+
+    int rowWidth = (int) memoryLimit;
+    int rowCount = 1;
+    int batchSize = rowWidth;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertTrue(memManager.getMergeMemoryLimit() < rowWidth);
+
+    // Only one spill batch, that batch is above the merge memory limit,
+    // but nothing useful comes from merging.
+
+    MergeTask task = memManager.consolidateBatches(0, 0, 1);
+    assertEquals(MergeAction.NONE, task.action);
   }
 
   @Test


[12/13] drill git commit: DRILL-5685: Provide a way to set common environment variable between sqlline and Drillbit differently.

Posted by jn...@apache.org.
DRILL-5685: Provide a way to set common environment variable between sqlline and Drillbit differently.

close #883


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/ae123e16
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/ae123e16
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/ae123e16

Branch: refs/heads/master
Commit: ae123e166af2ddb22c79b8cb93278a66fccc3be4
Parents: f1e1dfe
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Authored: Mon Jul 24 13:21:55 2017 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Mon Aug 14 22:19:24 2017 -0700

----------------------------------------------------------------------
 distribution/src/resources/drill-env.sh | 10 ++++++++++
 distribution/src/resources/drillbit.sh  |  5 +++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/ae123e16/distribution/src/resources/drill-env.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-env.sh b/distribution/src/resources/drill-env.sh
index 62dffe6..8211c3c 100644
--- a/distribution/src/resources/drill-env.sh
+++ b/distribution/src/resources/drill-env.sh
@@ -147,3 +147,13 @@
 # Set to "/tmp" by default.
 #
 # export DRILL_TMP_DIR="..."
+
+# Block to put environment variable known to both Sqlline and Drillbit, but needs to be
+# differently set for both. OR set for one and unset for other.
+#
+# if [ "$DRILLBIT_CONTEXT" = "1" ]; then
+#   Set environment variable value to be consumed by Drillbit
+# else
+#   Set environment variable value to be consumed by Sqlline
+# fi
+#

http://git-wip-us.apache.org/repos/asf/drill/blob/ae123e16/distribution/src/resources/drillbit.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drillbit.sh b/distribution/src/resources/drillbit.sh
index bba7392..de7f21a 100755
--- a/distribution/src/resources/drillbit.sh
+++ b/distribution/src/resources/drillbit.sh
@@ -64,6 +64,11 @@ bin=`cd -P "$bin">/dev/null; pwd`
 base=`basename "${BASH_SOURCE-$0}"`
 command=${base/.*/}
 
+# Environment variable to indicate Drillbit is being setup. Later drill-env.sh
+# and distrib-env.sh can consume this to set common environment variable differently
+# for Drillbit and Sqlline.
+export DRILLBIT_CONTEXT=1
+
 # Setup environment. This parses, and removes, the
 # options --config conf-dir parameters.
 


[10/13] drill git commit: DRILL-5699: Drill Web UI Page Source Has Links To External Sites

Posted by jn...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1cec10b9/exec/java-exec/src/main/resources/rest/static/js/jquery-3.2.1.min.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/jquery-3.2.1.min.js b/exec/java-exec/src/main/resources/rest/static/js/jquery-3.2.1.min.js
new file mode 100644
index 0000000..644d35e
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/jquery-3.2.1.min.js
@@ -0,0 +1,4 @@
+/*! jQuery v3.2.1 | (c) JS Foundation and other contributors | jquery.org/license */
+!function(a,b){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){"use strict";var c=[],d=a.document,e=Object.getPrototypeOf,f=c.slice,g=c.concat,h=c.push,i=c.indexOf,j={},k=j.toString,l=j.hasOwnProperty,m=l.toString,n=m.call(Object),o={};function p(a,b){b=b||d;var c=b.createElement("script");c.text=a,b.head.appendChild(c).parentNode.removeChild(c)}var q="3.2.1",r=function(a,b){return new r.fn.init(a,b)},s=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,t=/^-ms-/,u=/-([a-z])/g,v=function(a,b){return b.toUpperCase()};r.fn=r.prototype={jquery:q,constructor:r,length:0,toArray:function(){return f.call(this)},get:function(a){return null==a?f.call(this):a<0?this[a+this.length]:this[a]},pushStack:function(a){var b=r.merge(this.constructor(),a);return b.prevObject=this,b},each:function(a){retu
 rn r.each(this,a)},map:function(a){return this.pushStack(r.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(f.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(a<0?b:0);return this.pushStack(c>=0&&c<b?[this[c]]:[])},end:function(){return this.prevObject||this.constructor()},push:h,sort:c.sort,splice:c.splice},r.extend=r.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||r.isFunction(g)||(g={}),h===i&&(g=this,h--);h<i;h++)if(null!=(a=arguments[h]))for(b in a)c=g[b],d=a[b],g!==d&&(j&&d&&(r.isPlainObject(d)||(e=Array.isArray(d)))?(e?(e=!1,f=c&&Array.isArray(c)?c:[]):f=c&&r.isPlainObject(c)?c:{},g[b]=r.extend(j,f,d)):void 0!==d&&(g[b]=d));return g},r.extend({expando:"jQuery"+(q+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},no
 op:function(){},isFunction:function(a){return"function"===r.type(a)},isWindow:function(a){return null!=a&&a===a.window},isNumeric:function(a){var b=r.type(a);return("number"===b||"string"===b)&&!isNaN(a-parseFloat(a))},isPlainObject:function(a){var b,c;return!(!a||"[object Object]"!==k.call(a))&&(!(b=e(a))||(c=l.call(b,"constructor")&&b.constructor,"function"==typeof c&&m.call(c)===n))},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?j[k.call(a)]||"object":typeof a},globalEval:function(a){p(a)},camelCase:function(a){return a.replace(t,"ms-").replace(u,v)},each:function(a,b){var c,d=0;if(w(a)){for(c=a.length;d<c;d++)if(b.call(a[d],d,a[d])===!1)break}else for(d in a)if(b.call(a[d],d,a[d])===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(s,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(w(Object(a))?r.merge(c,"string"==typeof a?[a]:a):h.call(c,a)),c},inArray:funct
 ion(a,b,c){return null==b?-1:i.call(b,a,c)},merge:function(a,b){for(var c=+b.length,d=0,e=a.length;d<c;d++)a[e++]=b[d];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;f<g;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,e,f=0,h=[];if(w(a))for(d=a.length;f<d;f++)e=b(a[f],f,c),null!=e&&h.push(e);else for(f in a)e=b(a[f],f,c),null!=e&&h.push(e);return g.apply([],h)},guid:1,proxy:function(a,b){var c,d,e;if("string"==typeof b&&(c=a[b],b=a,a=c),r.isFunction(a))return d=f.call(arguments,2),e=function(){return a.apply(b||this,d.concat(f.call(arguments)))},e.guid=a.guid=a.guid||r.guid++,e},now:Date.now,support:o}),"function"==typeof Symbol&&(r.fn[Symbol.iterator]=c[Symbol.iterator]),r.each("Boolean Number String Function Array Date RegExp Object Error Symbol".split(" "),function(a,b){j["[object "+b+"]"]=b.toLowerCase()});function w(a){var b=!!a&&"length"in a&&a.length,c=r.type(a);return"function"!==c&&!r.isWindow(a)&&("array"===c||0===b||"
 number"==typeof b&&b>0&&b-1 in a)}var x=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ha(),z=ha(),A=ha(),B=function(a,b){return a===b&&(l=!0),0},C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=function(a,b){for(var c=0,d=a.length;c<d;c++)if(a[c]===b)return c;return-1},J="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",K="[\\x20\\t\\r\\n\\f]",L="(?:\\\\.|[\\w-]|[^\0-\\xa0])+",M="\\["+K+"*("+L+")(?:"+K+"*([*^$|!~]?=)"+K+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+L+"))|)"+K+"*\\]",N=":("+L+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+M+")*)|.*)\\)|)",O=new RegExp(K+"+","g"),P=new RegExp("^"+K+"+|((?:^|[^\\\\])(?:\\\\.)*)"+K+"+$","g"),Q=new RegExp("^"+K+"*,"+K+"*"),R=new RegExp("^"+K+"*([>+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(N),U=new RegExp("^
 "+L+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L+"|[*])"),ATTR:new RegExp("^"+M),PSEUDO:new RegExp("^"+N),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),aa=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:d<0?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},ba=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ca=function(a,b){return b?"\0"===a?"\ufffd":a.slice(0,-1)+"\\"+a.charCodeAt(a.length-1).toString(16)+" ":"\\"+a},da=function(){m()},ea=ta(function(a){retur
 n a.disabled===!0&&("form"in a||"label"in a)},{dir:"parentNode",next:"legend"});try{G.apply(D=H.call(v.childNodes),v.childNodes),D[v.childNodes.length].nodeType}catch(fa){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function ga(a,b,d,e){var f,h,j,k,l,o,r,s=b&&b.ownerDocument,w=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==w&&9!==w&&11!==w)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==w&&(l=Z.exec(a)))if(f=l[1]){if(9===w){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(s&&(j=s.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(l[2])return G.apply(d,b.getElementsByTagName(a)),d;if((f=l[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==w)s=b,r=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(ba,ca):b.set
 Attribute("id",k=u),o=g(a),h=o.length;while(h--)o[h]="#"+k+" "+sa(o[h]);r=o.join(","),s=$.test(a)&&qa(b.parentNode)||b}if(r)try{return G.apply(d,s.querySelectorAll(r)),d}catch(x){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(P,"$1"),b,d,e)}function ha(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ia(a){return a[u]=!0,a}function ja(a){var b=n.createElement("fieldset");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ka(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function la(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&a.sourceIndex-b.sourceIndex;if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function na(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}f
 unction oa(a){return function(b){return"form"in b?b.parentNode&&b.disabled===!1?"label"in b?"label"in b.parentNode?b.parentNode.disabled===a:b.disabled===a:b.isDisabled===a||b.isDisabled!==!a&&ea(b)===a:b.disabled===a:"label"in b&&b.disabled===a}}function pa(a){return ia(function(b){return b=+b,ia(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function qa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=ga.support={},f=ga.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return!!b&&"HTML"!==b.nodeName},m=ga.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),v!==n&&(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ja(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ja(function(a){return a.append
 Child(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Y.test(n.getElementsByClassName),c.getById=ja(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){return a.getAttribute("id")===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}}):(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c,d,e,f=b.getElementById(a);if(f){if(c=f.getAttributeNode("id"),c&&c.value===a)return[f];e=b.getElementsByName(a),d=0;while(f=e[d++])if(c=f.getAttributeNode("id"),c&&c.value===a)return[f]}return[]}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElement
 sByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){if("undefined"!=typeof b.getElementsByClassName&&p)return b.getElementsByClassName(a)},r=[],q=[],(c.qsa=Y.test(n.querySelectorAll))&&(ja(function(a){o.appendChild(a).innerHTML="<a id='"+u+"'></a><select id='"+u+"-\r\\' msallowcapture=''><option selected=''></option></select>",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ja(function(a){a.innerHTML="<a href='' disabled='disabled'></a><select disabled='disabled'><option/></select>";var b=n.createElement("input");b.setAttri
 bute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+K+"*[*^$|!~]?="),2!==a.querySelectorAll(":enabled").length&&q.push(":enabled",":disabled"),o.appendChild(a).disabled=!0,2!==a.querySelectorAll(":disabled").length&&q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Y.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ja(function(a){c.disconnectedMatch=s.call(a,"*"),s.call(a,"[s!='']:x"),r.push("!=",N)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Y.test(o.compareDocumentPosition),t=b||Y.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===
 b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?I(k,a)-I(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?I(k,a)-I(k,b):0;if(e===f)return la(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?la(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},ga.matches=function(a,b){return ga(a,null,null,b)},ga.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(S,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return ga(b,n,null,[a]).length>
 0},ga.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},ga.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},ga.escape=function(a){return(a+"").replace(ba,ca)},ga.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},ga.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=ga.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=ga.selectors={cacheLength:50,createPseudo:ia,match:V,attrHandle:{},find:{},
 relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(_,aa),a[3]=(a[3]||a[4]||a[5]||"").replace(_,aa),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||ga.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&ga.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return V.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&T.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(_,aa).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&y(a,function(a){return b.test("string"==typ
 eof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=ga.attr(d,a);return null==e?"!="===b:!b||(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(O," ")+" ").indexOf(c)>-1:"|="===b&&(e===c||e.slice(0,c.length+1)===c+"-"))}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=+
 +n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||ga.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ia(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ia(function(a){var b=[],c=[],d=h(a.replace(P,"$1"));return d[u]?ia(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ia(function(a){return function(b
 ){return ga(a,b).length>0}}),contains:ia(function(a){return a=a.replace(_,aa),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ia(function(a){return U.test(a||"")||ga.error("unsupported lang: "+a),a=a.replace(_,aa).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:oa(!1),disabled:oa(!0),checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:funct
 ion(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:pa(function(){return[0]}),last:pa(function(a,b){return[b-1]}),eq:pa(function(a,b,c){return[c<0?c+b:c]}),even:pa(function(a,b){for(var c=0;c<b;c+=2)a.push(c);return a}),odd:pa(function(a,b){for(var c=1;c<b;c+=2)a.push(c);return a}),lt:pa(function(a,b,c){for(var d=c<0?c+b:c;--d>=0;)a.push(d);return a}),gt:pa(function(a,b,c){for(var d=c<0?c+b:c;++d<b;)a.push(d);return a})}},d.pseudos.nth=d.pseudos.eq;for(b in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})d.pseudos[b]=ma(b);for(b in{submit:!0,reset:!0})d.pseudos[b]=na(b);function ra(){}ra.prototype=d.filters=d.pseudos,d.setFilters=new ra,g=ga.tokenize=function(a
 ,b){var c,e,f,g,h,i,j,k=z[a+" "];if(k)return b?0:k.slice(0);h=a,i=[],j=d.preFilter;while(h){c&&!(e=Q.exec(h))||(e&&(h=h.slice(e[0].length)||h),i.push(f=[])),c=!1,(e=R.exec(h))&&(c=e.shift(),f.push({value:c,type:e[0].replace(P," ")}),h=h.slice(c.length));for(g in d.filter)!(e=V[g].exec(h))||j[g]&&!(e=j[g](e))||(c=e.shift(),f.push({value:c,type:g,matches:e}),h=h.slice(c.length));if(!c)break}return b?h.length:h?ga.error(a):z(a,i).slice(0)};function sa(a){for(var b=0,c=a.length,d="";b<c;b++)d+=a[b].value;return d}function ta(a,b,c){var d=b.dir,e=b.next,f=e||d,g=c&&"parentNode"===f,h=x++;return b.first?function(b,c,e){while(b=b[d])if(1===b.nodeType||g)return a(b,c,e);return!1}:function(b,c,i){var j,k,l,m=[w,h];if(i){while(b=b[d])if((1===b.nodeType||g)&&a(b,c,i))return!0}else while(b=b[d])if(1===b.nodeType||g)if(l=b[u]||(b[u]={}),k=l[b.uniqueID]||(l[b.uniqueID]={}),e&&e===b.nodeName.toLowerCase())b=b[d]||b;else{if((j=k[f])&&j[0]===w&&j[1]===h)return m[2]=j[2];if(k[f]=m,m[2]=a(b,c,i))retur
 n!0}return!1}}function ua(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function va(a,b,c){for(var d=0,e=b.length;d<e;d++)ga(a,b[d],c);return c}function wa(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;h<i;h++)(f=a[h])&&(c&&!c(f,d,e)||(g.push(f),j&&b.push(h)));return g}function xa(a,b,c,d,e,f){return d&&!d[u]&&(d=xa(d)),e&&!e[u]&&(e=xa(e,f)),ia(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||va(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:wa(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=wa(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?I(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=wa(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ya(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ta(function(a){return a===b},
 h,!0),l=ta(function(a){return I(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];i<f;i++)if(c=d.relative[a[i].type])m=[ta(ua(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;e<f;e++)if(d.relative[a[e].type])break;return xa(i>1&&ua(m),i>1&&sa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(P,"$1"),c,i<e&&ya(a.slice(i,e)),e<f&&ya(a=a.slice(e)),e<f&&sa(a))}m.push(c)}return ua(m)}function za(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=E.call(i));u=wa(u)}G.apply(i,u),k&&!f&&u.length>0&&r+b.leng
 th>1&&ga.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ia(f):f}return h=ga.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=ya(b[c]),f[u]?d.push(f):e.push(f);f=A(a,za(e,d)),f.selector=a}return f},i=ga.select=function(a,b,c,e){var f,i,j,k,l,m="function"==typeof a&&a,n=!e&&g(a=m.selector||a);if(c=c||[],1===n.length){if(i=n[0]=n[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&9===b.nodeType&&p&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(_,aa),b)||[])[0],!b)return c;m&&(b=b.parentNode),a=a.slice(i.shift().value.length)}f=V.needsContext.test(a)?0:i.length;while(f--){if(j=i[f],d.relative[k=j.type])break;if((l=d.find[k])&&(e=l(j.matches[0].replace(_,aa),$.test(i[0].type)&&qa(b.parentNode)||b))){if(i.splice(f,1),a=e.length&&sa(i),!a)return G.apply(c,e),c;break}}}return(m||h(a,n))(e,b,!p,c,!b||$.test(a)&&qa(b.parentNode)||b),c},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ja(function(a){return
  1&a.compareDocumentPosition(n.createElement("fieldset"))}),ja(function(a){return a.innerHTML="<a href='#'></a>","#"===a.firstChild.getAttribute("href")})||ka("type|href|height|width",function(a,b,c){if(!c)return a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ja(function(a){return a.innerHTML="<input/>",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ka("value",function(a,b,c){if(!c&&"input"===a.nodeName.toLowerCase())return a.defaultValue}),ja(function(a){return null==a.getAttribute("disabled")})||ka(J,function(a,b,c){var d;if(!c)return a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),ga}(a);r.find=x,r.expr=x.selectors,r.expr[":"]=r.expr.pseudos,r.uniqueSort=r.unique=x.uniqueSort,r.text=x.getText,r.isXMLDoc=x.isXML,r.contains=x.contains,r.escapeSelector=x.escape;var y=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&r(a).is(c))break;d.push(a)}return d},z=func
 tion(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},A=r.expr.match.needsContext;function B(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()}var C=/^<([a-z][^\/\0>:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i,D=/^.[^:#\[\.,]*$/;function E(a,b,c){return r.isFunction(b)?r.grep(a,function(a,d){return!!b.call(a,d,a)!==c}):b.nodeType?r.grep(a,function(a){return a===b!==c}):"string"!=typeof b?r.grep(a,function(a){return i.call(b,a)>-1!==c}):D.test(b)?r.filter(b,a,c):(b=r.filter(b,a),r.grep(a,function(a){return i.call(b,a)>-1!==c&&1===a.nodeType}))}r.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?r.find.matchesSelector(d,a)?[d]:[]:r.find.matches(a,r.grep(b,function(a){return 1===a.nodeType}))},r.fn.extend({find:function(a){var b,c,d=this.length,e=this;if("string"!=typeof a)return this.pushStack(r(a).filter(function(){for(b=0;b<d;b++)if(r.contains(e[b],this))return!0}));for(c=this.pushStack([]),
 b=0;b<d;b++)r.find(a,e[b],c);return d>1?r.uniqueSort(c):c},filter:function(a){return this.pushStack(E(this,a||[],!1))},not:function(a){return this.pushStack(E(this,a||[],!0))},is:function(a){return!!E(this,"string"==typeof a&&A.test(a)?r(a):a||[],!1).length}});var F,G=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]+))$/,H=r.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||F,"string"==typeof a){if(e="<"===a[0]&&">"===a[a.length-1]&&a.length>=3?[null,a,null]:G.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof r?b[0]:b,r.merge(this,r.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),C.test(e[1])&&r.isPlainObject(b))for(e in b)r.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}return f=d.getElementById(e[2]),f&&(this[0]=f,this.length=1),this}return a.nodeType?(this[0]=a,this.length=1,this):r.isFunction(a)?void 0!==c.ready?c.ready(a):a(r):r.makeArray(a,this)};H.prototype=r.fn,F=r(d);var I=/^(?:parents|prev(?:Until|
 All))/,J={children:!0,contents:!0,next:!0,prev:!0};r.fn.extend({has:function(a){var b=r(a,this),c=b.length;return this.filter(function(){for(var a=0;a<c;a++)if(r.contains(this,b[a]))return!0})},closest:function(a,b){var c,d=0,e=this.length,f=[],g="string"!=typeof a&&r(a);if(!A.test(a))for(;d<e;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&r.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?r.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?i.call(r(a),this[0]):i.call(this,a.jquery?a[0]:a):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(r.uniqueSort(r.merge(this.get(),r(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function K(a,b){while((a=a[b])&&1!==a.nodeType);return a}r.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return y(a,"parentNode
 ")},parentsUntil:function(a,b,c){return y(a,"parentNode",c)},next:function(a){return K(a,"nextSibling")},prev:function(a){return K(a,"previousSibling")},nextAll:function(a){return y(a,"nextSibling")},prevAll:function(a){return y(a,"previousSibling")},nextUntil:function(a,b,c){return y(a,"nextSibling",c)},prevUntil:function(a,b,c){return y(a,"previousSibling",c)},siblings:function(a){return z((a.parentNode||{}).firstChild,a)},children:function(a){return z(a.firstChild)},contents:function(a){return B(a,"iframe")?a.contentDocument:(B(a,"template")&&(a=a.content||a),r.merge([],a.childNodes))}},function(a,b){r.fn[a]=function(c,d){var e=r.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=r.filter(d,e)),this.length>1&&(J[a]||r.uniqueSort(e),I.test(a)&&e.reverse()),this.pushStack(e)}});var L=/[^\x20\t\r\n\f]+/g;function M(a){var b={};return r.each(a.match(L)||[],function(a,c){b[c]=!0}),b}r.Callbacks=function(a){a="string"==typeof a?M(a):r.extend({},a);var b,c,d,e,f=
 [],g=[],h=-1,i=function(){for(e=e||a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h<f.length)f[h].apply(c[0],c[1])===!1&&a.stopOnFalse&&(h=f.length,c=!1)}a.memory||(c=!1),b=!1,e&&(f=c?[]:"")},j={add:function(){return f&&(c&&!b&&(h=f.length-1,g.push(c)),function d(b){r.each(b,function(b,c){r.isFunction(c)?a.unique&&j.has(c)||f.push(c):c&&c.length&&"string"!==r.type(c)&&d(c)})}(arguments),c&&!b&&i()),this},remove:function(){return r.each(arguments,function(a,b){var c;while((c=r.inArray(b,f,c))>-1)f.splice(c,1),c<=h&&h--}),this},has:function(a){return a?r.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=g=[],c||b||(f=c=""),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j};function N(a){return a}fu
 nction O(a){throw a}function P(a,b,c,d){var e;try{a&&r.isFunction(e=a.promise)?e.call(a).done(b).fail(c):a&&r.isFunction(e=a.then)?e.call(a,b,c):b.apply(void 0,[a].slice(d))}catch(a){c.apply(void 0,[a])}}r.extend({Deferred:function(b){var c=[["notify","progress",r.Callbacks("memory"),r.Callbacks("memory"),2],["resolve","done",r.Callbacks("once memory"),r.Callbacks("once memory"),0,"resolved"],["reject","fail",r.Callbacks("once memory"),r.Callbacks("once memory"),1,"rejected"]],d="pending",e={state:function(){return d},always:function(){return f.done(arguments).fail(arguments),this},"catch":function(a){return e.then(null,a)},pipe:function(){var a=arguments;return r.Deferred(function(b){r.each(c,function(c,d){var e=r.isFunction(a[d[4]])&&a[d[4]];f[d[1]](function(){var a=e&&e.apply(this,arguments);a&&r.isFunction(a.promise)?a.promise().progress(b.notify).done(b.resolve).fail(b.reject):b[d[0]+"With"](this,e?[a]:arguments)})}),a=null}).promise()},then:function(b,d,e){var f=0;function g(b
 ,c,d,e){return function(){var h=this,i=arguments,j=function(){var a,j;if(!(b<f)){if(a=d.apply(h,i),a===c.promise())throw new TypeError("Thenable self-resolution");j=a&&("object"==typeof a||"function"==typeof a)&&a.then,r.isFunction(j)?e?j.call(a,g(f,c,N,e),g(f,c,O,e)):(f++,j.call(a,g(f,c,N,e),g(f,c,O,e),g(f,c,N,c.notifyWith))):(d!==N&&(h=void 0,i=[a]),(e||c.resolveWith)(h,i))}},k=e?j:function(){try{j()}catch(a){r.Deferred.exceptionHook&&r.Deferred.exceptionHook(a,k.stackTrace),b+1>=f&&(d!==O&&(h=void 0,i=[a]),c.rejectWith(h,i))}};b?k():(r.Deferred.getStackHook&&(k.stackTrace=r.Deferred.getStackHook()),a.setTimeout(k))}}return r.Deferred(function(a){c[0][3].add(g(0,a,r.isFunction(e)?e:N,a.notifyWith)),c[1][3].add(g(0,a,r.isFunction(b)?b:N)),c[2][3].add(g(0,a,r.isFunction(d)?d:O))}).promise()},promise:function(a){return null!=a?r.extend(a,e):e}},f={};return r.each(c,function(a,b){var g=b[2],h=b[5];e[b[1]]=g.add,h&&g.add(function(){d=h},c[3-a][2].disable,c[0][2].lock),g.add(b[3].fire),
 f[b[0]]=function(){return f[b[0]+"With"](this===f?void 0:this,arguments),this},f[b[0]+"With"]=g.fireWith}),e.promise(f),b&&b.call(f,f),f},when:function(a){var b=arguments.length,c=b,d=Array(c),e=f.call(arguments),g=r.Deferred(),h=function(a){return function(c){d[a]=this,e[a]=arguments.length>1?f.call(arguments):c,--b||g.resolveWith(d,e)}};if(b<=1&&(P(a,g.done(h(c)).resolve,g.reject,!b),"pending"===g.state()||r.isFunction(e[c]&&e[c].then)))return g.then();while(c--)P(e[c],h(c),g.reject);return g.promise()}});var Q=/^(Eval|Internal|Range|Reference|Syntax|Type|URI)Error$/;r.Deferred.exceptionHook=function(b,c){a.console&&a.console.warn&&b&&Q.test(b.name)&&a.console.warn("jQuery.Deferred exception: "+b.message,b.stack,c)},r.readyException=function(b){a.setTimeout(function(){throw b})};var R=r.Deferred();r.fn.ready=function(a){return R.then(a)["catch"](function(a){r.readyException(a)}),this},r.extend({isReady:!1,readyWait:1,ready:function(a){(a===!0?--r.readyWait:r.isReady)||(r.isReady=!
 0,a!==!0&&--r.readyWait>0||R.resolveWith(d,[r]))}}),r.ready.then=R.then;function S(){d.removeEventListener("DOMContentLoaded",S),
+a.removeEventListener("load",S),r.ready()}"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll?a.setTimeout(r.ready):(d.addEventListener("DOMContentLoaded",S),a.addEventListener("load",S));var T=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===r.type(c)){e=!0;for(h in c)T(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,r.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(r(a),c)})),b))for(;h<i;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},U=function(a){return 1===a.nodeType||9===a.nodeType||!+a.nodeType};function V(){this.expando=r.expando+V.uid++}V.uid=1,V.prototype={cache:function(a){var b=a[this.expando];return b||(b={},U(a)&&(a.nodeType?a[this.expando]=b:Object.defineProperty(a,this.expando,{value:b,configurable:!0}))),b},set:function(a,b,c){var d,e=this.cache(a);if("string"==typeof b)e[r.camelCase(b)]=c;else for(d in b)e[r.camelCase(d)]=b[d];return e},get:function(a,
 b){return void 0===b?this.cache(a):a[this.expando]&&a[this.expando][r.camelCase(b)]},access:function(a,b,c){return void 0===b||b&&"string"==typeof b&&void 0===c?this.get(a,b):(this.set(a,b,c),void 0!==c?c:b)},remove:function(a,b){var c,d=a[this.expando];if(void 0!==d){if(void 0!==b){Array.isArray(b)?b=b.map(r.camelCase):(b=r.camelCase(b),b=b in d?[b]:b.match(L)||[]),c=b.length;while(c--)delete d[b[c]]}(void 0===b||r.isEmptyObject(d))&&(a.nodeType?a[this.expando]=void 0:delete a[this.expando])}},hasData:function(a){var b=a[this.expando];return void 0!==b&&!r.isEmptyObject(b)}};var W=new V,X=new V,Y=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,Z=/[A-Z]/g;function $(a){return"true"===a||"false"!==a&&("null"===a?null:a===+a+""?+a:Y.test(a)?JSON.parse(a):a)}function _(a,b,c){var d;if(void 0===c&&1===a.nodeType)if(d="data-"+b.replace(Z,"-$&").toLowerCase(),c=a.getAttribute(d),"string"==typeof c){try{c=$(c)}catch(e){}X.set(a,b,c)}else c=void 0;return c}r.extend({hasData:function(a){return X.hasData(a)|
 |W.hasData(a)},data:function(a,b,c){return X.access(a,b,c)},removeData:function(a,b){X.remove(a,b)},_data:function(a,b,c){return W.access(a,b,c)},_removeData:function(a,b){W.remove(a,b)}}),r.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=X.get(f),1===f.nodeType&&!W.get(f,"hasDataAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=r.camelCase(d.slice(5)),_(f,d,e[d])));W.set(f,"hasDataAttrs",!0)}return e}return"object"==typeof a?this.each(function(){X.set(this,a)}):T(this,function(b){var c;if(f&&void 0===b){if(c=X.get(f,a),void 0!==c)return c;if(c=_(f,a),void 0!==c)return c}else this.each(function(){X.set(this,a,b)})},null,b,arguments.length>1,null,!0)},removeData:function(a){return this.each(function(){X.remove(this,a)})}}),r.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=W.get(a,b),c&&(!d||Array.isArray(c)?d=W.access(a,b,r.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||
 "fx";var c=r.queue(a,b),d=c.length,e=c.shift(),f=r._queueHooks(a,b),g=function(){r.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return W.get(a,c)||W.access(a,c,{empty:r.Callbacks("once memory").add(function(){W.remove(a,[b+"queue",c])})})}}),r.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.length<c?r.queue(this[0],a):void 0===b?this:this.each(function(){var c=r.queue(this,a,b);r._queueHooks(this,a),"fx"===a&&"inprogress"!==c[0]&&r.dequeue(this,a)})},dequeue:function(a){return this.each(function(){r.dequeue(this,a)})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,b){var c,d=1,e=r.Deferred(),f=this,g=this.length,h=function(){--d||e.resolveWith(f,[f])};"string"!=typeof a&&(b=a,a=void 0),a=a||"fx";while(g--)c=W.get(f[g],a+"queueHooks"),c&&c.empty&&(d++,c.empty.add(h));return h
 (),e.promise(b)}});var aa=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,ba=new RegExp("^(?:([+-])=|)("+aa+")([a-z%]*)$","i"),ca=["Top","Right","Bottom","Left"],da=function(a,b){return a=b||a,"none"===a.style.display||""===a.style.display&&r.contains(a.ownerDocument,a)&&"none"===r.css(a,"display")},ea=function(a,b,c,d){var e,f,g={};for(f in b)g[f]=a.style[f],a.style[f]=b[f];e=c.apply(a,d||[]);for(f in b)a.style[f]=g[f];return e};function fa(a,b,c,d){var e,f=1,g=20,h=d?function(){return d.cur()}:function(){return r.css(a,b,"")},i=h(),j=c&&c[3]||(r.cssNumber[b]?"":"px"),k=(r.cssNumber[b]||"px"!==j&&+i)&&ba.exec(r.css(a,b));if(k&&k[3]!==j){j=j||k[3],c=c||[],k=+i||1;do f=f||".5",k/=f,r.style(a,b,k+j);while(f!==(f=h()/i)&&1!==f&&--g)}return c&&(k=+k||+i||0,e=c[1]?k+(c[1]+1)*c[2]:+c[2],d&&(d.unit=j,d.start=k,d.end=e)),e}var ga={};function ha(a){var b,c=a.ownerDocument,d=a.nodeName,e=ga[d];return e?e:(b=c.body.appendChild(c.createElement(d)),e=r.css(b,"display"),b.parentNode.removeChild(b),"
 none"===e&&(e="block"),ga[d]=e,e)}function ia(a,b){for(var c,d,e=[],f=0,g=a.length;f<g;f++)d=a[f],d.style&&(c=d.style.display,b?("none"===c&&(e[f]=W.get(d,"display")||null,e[f]||(d.style.display="")),""===d.style.display&&da(d)&&(e[f]=ha(d))):"none"!==c&&(e[f]="none",W.set(d,"display",c)));for(f=0;f<g;f++)null!=e[f]&&(a[f].style.display=e[f]);return a}r.fn.extend({show:function(){return ia(this,!0)},hide:function(){return ia(this)},toggle:function(a){return"boolean"==typeof a?a?this.show():this.hide():this.each(function(){da(this)?r(this).show():r(this).hide()})}});var ja=/^(?:checkbox|radio)$/i,ka=/<([a-z][^\/\0>\x20\t\r\n\f]+)/i,la=/^$|\/(?:java|ecma)script/i,ma={option:[1,"<select multiple='multiple'>","</select>"],thead:[1,"<table>","</table>"],col:[2,"<table><colgroup>","</colgroup></table>"],tr:[2,"<table><tbody>","</tbody></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:[0,"",""]};ma.optgroup=ma.option,ma.tbody=ma.tfoot=ma.colgroup=ma.caption=ma.thead,m
 a.th=ma.td;function na(a,b){var c;return c="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):[],void 0===b||b&&B(a,b)?r.merge([a],c):c}function oa(a,b){for(var c=0,d=a.length;c<d;c++)W.set(a[c],"globalEval",!b||W.get(b[c],"globalEval"))}var pa=/<|&#?\w+;/;function qa(a,b,c,d,e){for(var f,g,h,i,j,k,l=b.createDocumentFragment(),m=[],n=0,o=a.length;n<o;n++)if(f=a[n],f||0===f)if("object"===r.type(f))r.merge(m,f.nodeType?[f]:f);else if(pa.test(f)){g=g||l.appendChild(b.createElement("div")),h=(ka.exec(f)||["",""])[1].toLowerCase(),i=ma[h]||ma._default,g.innerHTML=i[1]+r.htmlPrefilter(f)+i[2],k=i[0];while(k--)g=g.lastChild;r.merge(m,g.childNodes),g=l.firstChild,g.textContent=""}else m.push(b.createTextNode(f));l.textContent="",n=0;while(f=m[n++])if(d&&r.inArray(f,d)>-1)e&&e.push(f);else if(j=r.contains(f.ownerDocument,f),g=na(l.appendChild(f),"script"),j&&oa(g),c){k=0;while(f=g[k++])la.test(f.type||""
 )&&c.push(f)}return l}!function(){var a=d.createDocumentFragment(),b=a.appendChild(d.createElement("div")),c=d.createElement("input");c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),b.appendChild(c),o.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,b.innerHTML="<textarea>x</textarea>",o.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue}();var ra=d.documentElement,sa=/^key/,ta=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,ua=/^([^.]*)(?:\.(.+)|)/;function va(){return!0}function wa(){return!1}function xa(){try{return d.activeElement}catch(a){}}function ya(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)ya(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=wa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return r().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=r.guid++)),a.each(
 function(){r.event.add(this,b,e,d,c)})}r.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.get(a);if(q){c.handler&&(f=c,c=f.handler,e=f.selector),e&&r.find.matchesSelector(ra,e),c.guid||(c.guid=r.guid++),(i=q.events)||(i=q.events={}),(g=q.handle)||(g=q.handle=function(b){return"undefined"!=typeof r&&r.event.triggered!==b.type?r.event.dispatch.apply(a,arguments):void 0}),b=(b||"").match(L)||[""],j=b.length;while(j--)h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n&&(l=r.event.special[n]||{},n=(e?l.delegateType:l.bindType)||n,l=r.event.special[n]||{},k=r.extend({type:n,origType:p,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&r.expr.match.needsContext.test(e),namespace:o.join(".")},f),(m=i[n])||(m=i[n]=[],m.delegateCount=0,l.setup&&l.setup.call(a,d,o,g)!==!1||a.addEventListener&&a.addEventListener(n,g)),l.add&&(l.add.call(a,k),k.handler.guid||(k.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,k):m.push(k),r.event.global[n]=!0)}},remove:f
 unction(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.hasData(a)&&W.get(a);if(q&&(i=q.events)){b=(b||"").match(L)||[""],j=b.length;while(j--)if(h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n){l=r.event.special[n]||{},n=(d?l.delegateType:l.bindType)||n,m=i[n]||[],h=h[2]&&new RegExp("(^|\\.)"+o.join("\\.(?:.*\\.|)")+"(\\.|$)"),g=f=m.length;while(f--)k=m[f],!e&&p!==k.origType||c&&c.guid!==k.guid||h&&!h.test(k.namespace)||d&&d!==k.selector&&("**"!==d||!k.selector)||(m.splice(f,1),k.selector&&m.delegateCount--,l.remove&&l.remove.call(a,k));g&&!m.length&&(l.teardown&&l.teardown.call(a,o,q.handle)!==!1||r.removeEvent(a,n,q.handle),delete i[n])}else for(n in i)r.event.remove(a,n+b[j],c,d,!0);r.isEmptyObject(i)&&W.remove(a,"handle events")}},dispatch:function(a){var b=r.event.fix(a),c,d,e,f,g,h,i=new Array(arguments.length),j=(W.get(this,"events")||{})[b.type]||[],k=r.event.special[b.type]||{};for(i[0]=b,c=1;c<arguments.length;c++)i[c]=arguments[c];if(b.delegateTarget=this,!k.preDi
 spatch||k.preDispatch.call(this,b)!==!1){h=r.event.handlers.call(this,b,j),c=0;while((f=h[c++])&&!b.isPropagationStopped()){b.currentTarget=f.elem,d=0;while((g=f.handlers[d++])&&!b.isImmediatePropagationStopped())b.rnamespace&&!b.rnamespace.test(g.namespace)||(b.handleObj=g,b.data=g.data,e=((r.event.special[g.origType]||{}).handle||g.handler).apply(f.elem,i),void 0!==e&&(b.result=e)===!1&&(b.preventDefault(),b.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,b),b.result}},handlers:function(a,b){var c,d,e,f,g,h=[],i=b.delegateCount,j=a.target;if(i&&j.nodeType&&!("click"===a.type&&a.button>=1))for(;j!==this;j=j.parentNode||this)if(1===j.nodeType&&("click"!==a.type||j.disabled!==!0)){for(f=[],g={},c=0;c<i;c++)d=b[c],e=d.selector+" ",void 0===g[e]&&(g[e]=d.needsContext?r(e,this).index(j)>-1:r.find(e,this,null,[j]).length),g[e]&&f.push(d);f.length&&h.push({elem:j,handlers:f})}return j=this,i<b.length&&h.push({elem:j,handlers:b.slice(i)}),h},addProp:function(a,b){Object
 .defineProperty(r.Event.prototype,a,{enumerable:!0,configurable:!0,get:r.isFunction(b)?function(){if(this.originalEvent)return b(this.originalEvent)}:function(){if(this.originalEvent)return this.originalEvent[a]},set:function(b){Object.defineProperty(this,a,{enumerable:!0,configurable:!0,writable:!0,value:b})}})},fix:function(a){return a[r.expando]?a:new r.Event(a)},special:{load:{noBubble:!0},focus:{trigger:function(){if(this!==xa()&&this.focus)return this.focus(),!1},delegateType:"focusin"},blur:{trigger:function(){if(this===xa()&&this.blur)return this.blur(),!1},delegateType:"focusout"},click:{trigger:function(){if("checkbox"===this.type&&this.click&&B(this,"input"))return this.click(),!1},_default:function(a){return B(a.target,"a")}},beforeunload:{postDispatch:function(a){void 0!==a.result&&a.originalEvent&&(a.originalEvent.returnValue=a.result)}}}},r.removeEvent=function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c)},r.Event=function(a,b){return this instanceof r.Eve
 nt?(a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||void 0===a.defaultPrevented&&a.returnValue===!1?va:wa,this.target=a.target&&3===a.target.nodeType?a.target.parentNode:a.target,this.currentTarget=a.currentTarget,this.relatedTarget=a.relatedTarget):this.type=a,b&&r.extend(this,b),this.timeStamp=a&&a.timeStamp||r.now(),void(this[r.expando]=!0)):new r.Event(a,b)},r.Event.prototype={constructor:r.Event,isDefaultPrevented:wa,isPropagationStopped:wa,isImmediatePropagationStopped:wa,isSimulated:!1,preventDefault:function(){var a=this.originalEvent;this.isDefaultPrevented=va,a&&!this.isSimulated&&a.preventDefault()},stopPropagation:function(){var a=this.originalEvent;this.isPropagationStopped=va,a&&!this.isSimulated&&a.stopPropagation()},stopImmediatePropagation:function(){var a=this.originalEvent;this.isImmediatePropagationStopped=va,a&&!this.isSimulated&&a.stopImmediatePropagation(),this.stopPropagation()}},r.each({altKey:!0,bubbles:!0,cancel
 able:!0,changedTouches:!0,ctrlKey:!0,detail:!0,eventPhase:!0,metaKey:!0,pageX:!0,pageY:!0,shiftKey:!0,view:!0,"char":!0,charCode:!0,key:!0,keyCode:!0,button:!0,buttons:!0,clientX:!0,clientY:!0,offsetX:!0,offsetY:!0,pointerId:!0,pointerType:!0,screenX:!0,screenY:!0,targetTouches:!0,toElement:!0,touches:!0,which:function(a){var b=a.button;return null==a.which&&sa.test(a.type)?null!=a.charCode?a.charCode:a.keyCode:!a.which&&void 0!==b&&ta.test(a.type)?1&b?1:2&b?3:4&b?2:0:a.which}},r.event.addProp),r.each({mouseenter:"mouseover",mouseleave:"mouseout",pointerenter:"pointerover",pointerleave:"pointerout"},function(a,b){r.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj;return e&&(e===d||r.contains(d,e))||(a.type=f.origType,c=f.handler.apply(this,arguments),a.type=b),c}}}),r.fn.extend({on:function(a,b,c,d){return ya(this,a,b,c,d)},one:function(a,b,c,d){return ya(this,a,b,c,d,1)},off:function(a,b,c){var d,e;if(a&&a.preventDefault&&a
 .handleObj)return d=a.handleObj,r(a.delegateTarget).off(d.namespace?d.origType+"."+d.namespace:d.origType,d.selector,d.handler),this;if("object"==typeof a){for(e in a)this.off(e,b,a[e]);return this}return b!==!1&&"function"!=typeof b||(c=b,b=void 0),c===!1&&(c=wa),this.each(function(){r.event.remove(this,a,c,b)})}});var za=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([a-z][^\/\0>\x20\t\r\n\f]*)[^>]*)\/>/gi,Aa=/<script|<style|<link/i,Ba=/checked\s*(?:[^=]|=\s*.checked.)/i,Ca=/^true\/(.*)/,Da=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g;function Ea(a,b){return B(a,"table")&&B(11!==b.nodeType?b:b.firstChild,"tr")?r(">tbody",a)[0]||a:a}function Fa(a){return a.type=(null!==a.getAttribute("type"))+"/"+a.type,a}function Ga(a){var b=Ca.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Ha(a,b){var c,d,e,f,g,h,i,j;if(1===b.nodeType){if(W.hasData(a)&&(f=W.access(a),g=W.set(b,f),j=f.events)){delete g.handle,g.events={};for(e in j)for(c=0,d=j[e].length;c<d;c++)r.event
 .add(b,e,j[e][c])}X.hasData(a)&&(h=X.access(a),i=r.extend({},h),X.set(b,i))}}function Ia(a,b){var c=b.nodeName.toLowerCase();"input"===c&&ja.test(a.type)?b.checked=a.checked:"input"!==c&&"textarea"!==c||(b.defaultValue=a.defaultValue)}function Ja(a,b,c,d){b=g.apply([],b);var e,f,h,i,j,k,l=0,m=a.length,n=m-1,q=b[0],s=r.isFunction(q);if(s||m>1&&"string"==typeof q&&!o.checkClone&&Ba.test(q))return a.each(function(e){var f=a.eq(e);s&&(b[0]=q.call(this,e,f.html())),Ja(f,b,c,d)});if(m&&(e=qa(b,a[0].ownerDocument,!1,a,d),f=e.firstChild,1===e.childNodes.length&&(e=f),f||d)){for(h=r.map(na(e,"script"),Fa),i=h.length;l<m;l++)j=e,l!==n&&(j=r.clone(j,!0,!0),i&&r.merge(h,na(j,"script"))),c.call(a[l],j,l);if(i)for(k=h[h.length-1].ownerDocument,r.map(h,Ga),l=0;l<i;l++)j=h[l],la.test(j.type||"")&&!W.access(j,"globalEval")&&r.contains(k,j)&&(j.src?r._evalUrl&&r._evalUrl(j.src):p(j.textContent.replace(Da,""),k))}return a}function Ka(a,b,c){for(var d,e=b?r.filter(b,a):a,f=0;null!=(d=e[f]);f++)c||1!==d
 .nodeType||r.cleanData(na(d)),d.parentNode&&(c&&r.contains(d.ownerDocument,d)&&oa(na(d,"script")),d.parentNode.removeChild(d));return a}r.extend({htmlPrefilter:function(a){return a.replace(za,"<$1></$2>")},clone:function(a,b,c){var d,e,f,g,h=a.cloneNode(!0),i=r.contains(a.ownerDocument,a);if(!(o.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||r.isXMLDoc(a)))for(g=na(h),f=na(a),d=0,e=f.length;d<e;d++)Ia(f[d],g[d]);if(b)if(c)for(f=f||na(a),g=g||na(h),d=0,e=f.length;d<e;d++)Ha(f[d],g[d]);else Ha(a,h);return g=na(h,"script"),g.length>0&&oa(g,!i&&na(a,"script")),h},cleanData:function(a){for(var b,c,d,e=r.event.special,f=0;void 0!==(c=a[f]);f++)if(U(c)){if(b=c[W.expando]){if(b.events)for(d in b.events)e[d]?r.event.remove(c,d):r.removeEvent(c,d,b.handle);c[W.expando]=void 0}c[X.expando]&&(c[X.expando]=void 0)}}}),r.fn.extend({detach:function(a){return Ka(this,a,!0)},remove:function(a){return Ka(this,a)},text:function(a){return T(this,function(a){return void 0===a?r.text(this):this.empty()
 .each(function(){1!==this.nodeType&&11!==this.nodeType&&9!==this.nodeType||(this.textContent=a)})},null,a,arguments.length)},append:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.appendChild(a)}})},prepend:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},empty:function(){for(var a,b=0;null!=(a=this[b]);b++)1===a.nodeType&&(r.cleanData(na(a,!1)),a.textContent="");return this},clone:function(a,b){return a=null!=a&&a,b=null==b?a:b,this.map(function(){return r.clone(this,a,b)})},html:function(a){return T(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0=
 ==a&&1===b.nodeType)return b.innerHTML;if("string"==typeof a&&!Aa.test(a)&&!ma[(ka.exec(a)||["",""])[1].toLowerCase()]){a=r.htmlPrefilter(a);try{for(;c<d;c++)b=this[c]||{},1===b.nodeType&&(r.cleanData(na(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=[];return Ja(this,arguments,function(b){var c=this.parentNode;r.inArray(this,a)<0&&(r.cleanData(na(this)),c&&c.replaceChild(b,this))},a)}}),r.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){r.fn[a]=function(a){for(var c,d=[],e=r(a),f=e.length-1,g=0;g<=f;g++)c=g===f?this:this.clone(!0),r(e[g])[b](c),h.apply(d,c.get());return this.pushStack(d)}});var La=/^margin/,Ma=new RegExp("^("+aa+")(?!px)[a-z%]+$","i"),Na=function(b){var c=b.ownerDocument.defaultView;return c&&c.opener||(c=a),c.getComputedStyle(b)};!function(){function b(){if(i){i.style.cssText="box-sizing:border-box;position:relative;d
 isplay:block;margin:auto;border:1px;padding:1px;top:1%;width:50%",i.innerHTML="",ra.appendChild(h);var b=a.getComputedStyle(i);c="1%"!==b.top,g="2px"===b.marginLeft,e="4px"===b.width,i.style.marginRight="50%",f="4px"===b.marginRight,ra.removeChild(h),i=null}}var c,e,f,g,h=d.createElement("div"),i=d.createElement("div");i.style&&(i.style.backgroundClip="content-box",i.cloneNode(!0).style.backgroundClip="",o.clearCloneStyle="content-box"===i.style.backgroundClip,h.style.cssText="border:0;width:8px;height:0;top:0;left:-9999px;padding:0;margin-top:1px;position:absolute",h.appendChild(i),r.extend(o,{pixelPosition:function(){return b(),c},boxSizingReliable:function(){return b(),e},pixelMarginRight:function(){return b(),f},reliableMarginLeft:function(){return b(),g}}))}();function Oa(a,b,c){var d,e,f,g,h=a.style;return c=c||Na(a),c&&(g=c.getPropertyValue(b)||c[b],""!==g||r.contains(a.ownerDocument,a)||(g=r.style(a,b)),!o.pixelMarginRight()&&Ma.test(g)&&La.test(b)&&(d=h.width,e=h.minWidth,f
 =h.maxWidth,h.minWidth=h.maxWidth=h.width=g,g=c.width,h.width=d,h.minWidth=e,h.maxWidth=f)),void 0!==g?g+"":g}function Pa(a,b){return{get:function(){return a()?void delete this.get:(this.get=b).apply(this,arguments)}}}var Qa=/^(none|table(?!-c[ea]).+)/,Ra=/^--/,Sa={position:"absolute",visibility:"hidden",display:"block"},Ta={letterSpacing:"0",fontWeight:"400"},Ua=["Webkit","Moz","ms"],Va=d.createElement("div").style;function Wa(a){if(a in Va)return a;var b=a[0].toUpperCase()+a.slice(1),c=Ua.length;while(c--)if(a=Ua[c]+b,a in Va)return a}function Xa(a){var b=r.cssProps[a];return b||(b=r.cssProps[a]=Wa(a)||a),b}function Ya(a,b,c){var d=ba.exec(b);return d?Math.max(0,d[2]-(c||0))+(d[3]||"px"):b}function Za(a,b,c,d,e){var f,g=0;for(f=c===(d?"border":"content")?4:"width"===b?1:0;f<4;f+=2)"margin"===c&&(g+=r.css(a,c+ca[f],!0,e)),d?("content"===c&&(g-=r.css(a,"padding"+ca[f],!0,e)),"margin"!==c&&(g-=r.css(a,"border"+ca[f]+"Width",!0,e))):(g+=r.css(a,"padding"+ca[f],!0,e),"padding"!==c&&(g+
 =r.css(a,"border"+ca[f]+"Width",!0,e)));return g}function $a(a,b,c){var d,e=Na(a),f=Oa(a,b,e),g="border-box"===r.css(a,"boxSizing",!1,e);return Ma.test(f)?f:(d=g&&(o.boxSizingReliable()||f===a.style[b]),"auto"===f&&(f=a["offset"+b[0].toUpperCase()+b.slice(1)]),f=parseFloat(f)||0,f+Za(a,b,c||(g?"border":"content"),d,e)+"px")}r.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=Oa(a,"opacity");return""===c?"1":c}}}},cssNumber:{animationIterationCount:!0,columnCount:!0,fillOpacity:!0,flexGrow:!0,flexShrink:!0,fontWeight:!0,lineHeight:!0,opacity:!0,order:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":"cssFloat"},style:function(a,b,c,d){if(a&&3!==a.nodeType&&8!==a.nodeType&&a.style){var e,f,g,h=r.camelCase(b),i=Ra.test(b),j=a.style;return i||(b=Xa(h)),g=r.cssHooks[b]||r.cssHooks[h],void 0===c?g&&"get"in g&&void 0!==(e=g.get(a,!1,d))?e:j[b]:(f=typeof c,"string"===f&&(e=ba.exec(c))&&e[1]&&(c=fa(a,b,e),f="number"),null!=c&&c===c&&("number"===f&&(c+=e&&e[3]||(r.cssNumber[
 h]?"":"px")),o.clearCloneStyle||""!==c||0!==b.indexOf("background")||(j[b]="inherit"),g&&"set"in g&&void 0===(c=g.set(a,c,d))||(i?j.setProperty(b,c):j[b]=c)),void 0)}},css:function(a,b,c,d){var e,f,g,h=r.camelCase(b),i=Ra.test(b);return i||(b=Xa(h)),g=r.cssHooks[b]||r.cssHooks[h],g&&"get"in g&&(e=g.get(a,!0,c)),void 0===e&&(e=Oa(a,b,d)),"normal"===e&&b in Ta&&(e=Ta[b]),""===c||c?(f=parseFloat(e),c===!0||isFinite(f)?f||0:e):e}}),r.each(["height","width"],function(a,b){r.cssHooks[b]={get:function(a,c,d){if(c)return!Qa.test(r.css(a,"display"))||a.getClientRects().length&&a.getBoundingClientRect().width?$a(a,b,d):ea(a,Sa,function(){return $a(a,b,d)})},set:function(a,c,d){var e,f=d&&Na(a),g=d&&Za(a,b,d,"border-box"===r.css(a,"boxSizing",!1,f),f);return g&&(e=ba.exec(c))&&"px"!==(e[3]||"px")&&(a.style[b]=c,c=r.css(a,b)),Ya(a,c,g)}}}),r.cssHooks.marginLeft=Pa(o.reliableMarginLeft,function(a,b){if(b)return(parseFloat(Oa(a,"marginLeft"))||a.getBoundingClientRect().left-ea(a,{marginLeft:0},fu
 nction(){return a.getBoundingClientRect().left}))+"px"}),r.each({margin:"",padding:"",border:"Width"},function(a,b){r.cssHooks[a+b]={expand:function(c){for(var d=0,e={},f="string"==typeof c?c.split(" "):[c];d<4;d++)e[a+ca[d]+b]=f[d]||f[d-2]||f[0];return e}},La.test(a)||(r.cssHooks[a+b].set=Ya)}),r.fn.extend({css:function(a,b){return T(this,function(a,b,c){var d,e,f={},g=0;if(Array.isArray(b)){for(d=Na(a),e=b.length;g<e;g++)f[b[g]]=r.css(a,b[g],!1,d);return f}return void 0!==c?r.style(a,b,c):r.css(a,b)},a,b,arguments.length>1)}});function _a(a,b,c,d,e){return new _a.prototype.init(a,b,c,d,e)}r.Tween=_a,_a.prototype={constructor:_a,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||r.easing._default,this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(r.cssNumber[c]?"":"px")},cur:function(){var a=_a.propHooks[this.prop];return a&&a.get?a.get(this):_a.propHooks._default.get(this)},run:function(a){var b,c=_a.propHooks[this.prop];return this.options.durat
 ion?this.pos=b=r.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):this.pos=b=a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):_a.propHooks._default.set(this),this}},_a.prototype.init.prototype=_a.prototype,_a.propHooks={_default:{get:function(a){var b;return 1!==a.elem.nodeType||null!=a.elem[a.prop]&&null==a.elem.style[a.prop]?a.elem[a.prop]:(b=r.css(a.elem,a.prop,""),b&&"auto"!==b?b:0)},set:function(a){r.fx.step[a.prop]?r.fx.step[a.prop](a):1!==a.elem.nodeType||null==a.elem.style[r.cssProps[a.prop]]&&!r.cssHooks[a.prop]?a.elem[a.prop]=a.now:r.style(a.elem,a.prop,a.now+a.unit)}}},_a.propHooks.scrollTop=_a.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},r.easing={linear:function(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2},_default:"swing"},r.fx=_a.prototype.init,r.fx.step={};var ab,bb,cb=/^(?:toggle|show|hid
 e)$/,db=/queueHooks$/;function eb(){bb&&(d.hidden===!1&&a.requestAnimationFrame?a.requestAnimationFrame(eb):a.setTimeout(eb,r.fx.interval),r.fx.tick())}function fb(){return a.setTimeout(function(){ab=void 0}),ab=r.now()}function gb(a,b){var c,d=0,e={height:a};for(b=b?1:0;d<4;d+=2-b)c=ca[d],e["margin"+c]=e["padding"+c]=a;return b&&(e.opacity=e.width=a),e}function hb(a,b,c){for(var d,e=(kb.tweeners[b]||[]).concat(kb.tweeners["*"]),f=0,g=e.length;f<g;f++)if(d=e[f].call(c,b,a))return d}function ib(a,b,c){var d,e,f,g,h,i,j,k,l="width"in b||"height"in b,m=this,n={},o=a.style,p=a.nodeType&&da(a),q=W.get(a,"fxshow");c.queue||(g=r._queueHooks(a,"fx"),null==g.unqueued&&(g.unqueued=0,h=g.empty.fire,g.empty.fire=function(){g.unqueued||h()}),g.unqueued++,m.always(function(){m.always(function(){g.unqueued--,r.queue(a,"fx").length||g.empty.fire()})}));for(d in b)if(e=b[d],cb.test(e)){if(delete b[d],f=f||"toggle"===e,e===(p?"hide":"show")){if("show"!==e||!q||void 0===q[d])continue;p=!0}n[d]=q&&q[d]
 ||r.style(a,d)}if(i=!r.isEmptyObject(b),i||!r.isEmptyObject(n)){l&&1===a.nodeType&&(c.overflow=[o.overflow,o.overflowX,o.overflowY],j=q&&q.display,null==j&&(j=W.get(a,"display")),k=r.css(a,"display"),"none"===k&&(j?k=j:(ia([a],!0),j=a.style.display||j,k=r.css(a,"display"),ia([a]))),("inline"===k||"inline-block"===k&&null!=j)&&"none"===r.css(a,"float")&&(i||(m.done(function(){o.display=j}),null==j&&(k=o.display,j="none"===k?"":k)),o.display="inline-block")),c.overflow&&(o.overflow="hidden",m.always(function(){o.overflow=c.overflow[0],o.overflowX=c.overflow[1],o.overflowY=c.overflow[2]})),i=!1;for(d in n)i||(q?"hidden"in q&&(p=q.hidden):q=W.access(a,"fxshow",{display:j}),f&&(q.hidden=!p),p&&ia([a],!0),m.done(function(){p||ia([a]),W.remove(a,"fxshow");for(d in n)r.style(a,d,n[d])})),i=hb(p?q[d]:0,d,m),d in q||(q[d]=i.start,p&&(i.end=i.start,i.start=0))}}function jb(a,b){var c,d,e,f,g;for(c in a)if(d=r.camelCase(c),e=b[d],f=a[c],Array.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,dele
 te a[c]),g=r.cssHooks[d],g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}function kb(a,b,c){var d,e,f=0,g=kb.prefilters.length,h=r.Deferred().always(function(){delete i.elem}),i=function(){if(e)return!1;for(var b=ab||fb(),c=Math.max(0,j.startTime+j.duration-b),d=c/j.duration||0,f=1-d,g=0,i=j.tweens.length;g<i;g++)j.tweens[g].run(f);return h.notifyWith(a,[j,f,c]),f<1&&i?c:(i||h.notifyWith(a,[j,1,0]),h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:r.extend({},b),opts:r.extend(!0,{specialEasing:{},easing:r.easing._default},c),originalProperties:b,originalOptions:c,startTime:ab||fb(),duration:c.duration,tweens:[],createTween:function(b,c){var d=r.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(d),d},stop:function(b){var c=0,d=b?j.tweens.length:0;if(e)return this;for(e=!0;c<d;c++)j.tweens[c].run(1);return b?(h.notifyWith(a,[j,1,0]),h.resolveWith(a,[j,b])):h.rejectWith(a,[j,b]),this}}),k=j.props;for(jb(k,j.op
 ts.specialEasing);f<g;f++)if(d=kb.prefilters[f].call(j,a,k,j.opts))return r.isFunction(d.stop)&&(r._queueHooks(j.elem,j.opts.queue).stop=r.proxy(d.stop,d)),d;return r.map(k,hb,j),r.isFunction(j.opts.start)&&j.opts.start.call(a,j),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(j.opts.always),r.fx.timer(r.extend(i,{elem:a,anim:j,queue:j.opts.queue})),j}r.Animation=r.extend(kb,{tweeners:{"*":[function(a,b){var c=this.createTween(a,b);return fa(c.elem,a,ba.exec(b),c),c}]},tweener:function(a,b){r.isFunction(a)?(b=a,a=["*"]):a=a.match(L);for(var c,d=0,e=a.length;d<e;d++)c=a[d],kb.tweeners[c]=kb.tweeners[c]||[],kb.tweeners[c].unshift(b)},prefilters:[ib],prefilter:function(a,b){b?kb.prefilters.unshift(a):kb.prefilters.push(a)}}),r.speed=function(a,b,c){var d=a&&"object"==typeof a?r.extend({},a):{complete:c||!c&&b||r.isFunction(a)&&a,duration:a,easing:c&&b||b&&!r.isFunction(b)&&b};return r.fx.off?d.duration=0:"number"!=typeof d.duration&&(d.duration in
  r.fx.speeds?d.duration=r.fx.speeds[d.duration]:d.duration=r.fx.speeds._default),null!=d.queue&&d.queue!==!0||(d.queue="fx"),d.old=d.complete,d.complete=function(){r.isFunction(d.old)&&d.old.call(this),d.queue&&r.dequeue(this,d.queue)},d},r.fn.extend({fadeTo:function(a,b,c,d){return this.filter(da).css("opacity",0).show().end().animate({opacity:b},a,c,d)},animate:function(a,b,c,d){var e=r.isEmptyObject(a),f=r.speed(b,c,d),g=function(){var b=kb(this,r.extend({},a),f);(e||W.get(this,"finish"))&&b.stop(!0)};return g.finish=g,e||f.queue===!1?this.each(g):this.queue(f.queue,g)},stop:function(a,b,c){var d=function(a){var b=a.stop;delete a.stop,b(c)};return"string"!=typeof a&&(c=b,b=a,a=void 0),b&&a!==!1&&this.queue(a||"fx",[]),this.each(function(){var b=!0,e=null!=a&&a+"queueHooks",f=r.timers,g=W.get(this);if(e)g[e]&&g[e].stop&&d(g[e]);else for(e in g)g[e]&&g[e].stop&&db.test(e)&&d(g[e]);for(e=f.length;e--;)f[e].elem!==this||null!=a&&f[e].queue!==a||(f[e].anim.stop(c),b=!1,f.splice(e,1));
 !b&&c||r.dequeue(this,a)})},finish:function(a){return a!==!1&&(a=a||"fx"),this.each(function(){var b,c=W.get(this),d=c[a+"queue"],e=c[a+"queueHooks"],f=r.timers,g=d?d.length:0;for(c.finish=!0,r.queue(this,a,[]),e&&e.stop&&e.stop.call(this,!0),b=f.length;b--;)f[b].elem===this&&f[b].queue===a&&(f[b].anim.stop(!0),f.splice(b,1));for(b=0;b<g;b++)d[b]&&d[b].finish&&d[b].finish.call(this);delete c.finish})}}),r.each(["toggle","show","hide"],function(a,b){var c=r.fn[b];r.fn[b]=function(a,d,e){return null==a||"boolean"==typeof a?c.apply(this,arguments):this.animate(gb(b,!0),a,d,e)}}),r.each({slideDown:gb("show"),slideUp:gb("hide"),slideToggle:gb("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(a,b){r.fn[a]=function(a,c,d){return this.animate(b,a,c,d)}}),r.timers=[],r.fx.tick=function(){var a,b=0,c=r.timers;for(ab=r.now();b<c.length;b++)a=c[b],a()||c[b]!==a||c.splice(b--,1);c.length||r.fx.stop(),ab=void 0},r.fx.timer=function(a){r.timers.push
 (a),r.fx.start()},r.fx.interval=13,r.fx.start=function(){bb||(bb=!0,eb())},r.fx.stop=function(){bb=null},r.fx.speeds={slow:600,fast:200,_default:400},r.fn.delay=function(b,c){return b=r.fx?r.fx.speeds[b]||b:b,c=c||"fx",this.queue(c,function(c,d){var e=a.setTimeout(c,b);d.stop=function(){a.clearTimeout(e)}})},function(){var a=d.createElement("input"),b=d.createElement("select"),c=b.appendChild(d.createElement("option"));a.type="checkbox",o.checkOn=""!==a.value,o.optSelected=c.selected,a=d.createElement("input"),a.value="t",a.type="radio",o.radioValue="t"===a.value}();var lb,mb=r.expr.attrHandle;r.fn.extend({attr:function(a,b){return T(this,r.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){r.removeAttr(this,a)})}}),r.extend({attr:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return"undefined"==typeof a.getAttribute?r.prop(a,b,c):(1===f&&r.isXMLDoc(a)||(e=r.attrHooks[b.toLowerCase()]||(r.expr.match.bool.test(b)?lb:void 0)),void 0!==c?null=
 ==c?void r.removeAttr(a,b):e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:(a.setAttribute(b,c+""),c):e&&"get"in e&&null!==(d=e.get(a,b))?d:(d=r.find.attr(a,b),
+null==d?void 0:d))},attrHooks:{type:{set:function(a,b){if(!o.radioValue&&"radio"===b&&B(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}}},removeAttr:function(a,b){var c,d=0,e=b&&b.match(L);if(e&&1===a.nodeType)while(c=e[d++])a.removeAttribute(c)}}),lb={set:function(a,b,c){return b===!1?r.removeAttr(a,c):a.setAttribute(c,c),c}},r.each(r.expr.match.bool.source.match(/\w+/g),function(a,b){var c=mb[b]||r.find.attr;mb[b]=function(a,b,d){var e,f,g=b.toLowerCase();return d||(f=mb[g],mb[g]=e,e=null!=c(a,b,d)?g:null,mb[g]=f),e}});var nb=/^(?:input|select|textarea|button)$/i,ob=/^(?:a|area)$/i;r.fn.extend({prop:function(a,b){return T(this,r.prop,a,b,arguments.length>1)},removeProp:function(a){return this.each(function(){delete this[r.propFix[a]||a]})}}),r.extend({prop:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return 1===f&&r.isXMLDoc(a)||(b=r.propFix[b]||b,e=r.propHooks[b]),void 0!==c?e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:a[b]=c:e&&"get"in e&
 &null!==(d=e.get(a,b))?d:a[b]},propHooks:{tabIndex:{get:function(a){var b=r.find.attr(a,"tabindex");return b?parseInt(b,10):nb.test(a.nodeName)||ob.test(a.nodeName)&&a.href?0:-1}}},propFix:{"for":"htmlFor","class":"className"}}),o.optSelected||(r.propHooks.selected={get:function(a){var b=a.parentNode;return b&&b.parentNode&&b.parentNode.selectedIndex,null},set:function(a){var b=a.parentNode;b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex)}}),r.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"],function(){r.propFix[this.toLowerCase()]=this});function pb(a){var b=a.match(L)||[];return b.join(" ")}function qb(a){return a.getAttribute&&a.getAttribute("class")||""}r.fn.extend({addClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).addClass(a.call(this,b,qb(this)))});if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&
 &" "+pb(e)+" "){g=0;while(f=b[g++])d.indexOf(" "+f+" ")<0&&(d+=f+" ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},removeClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).removeClass(a.call(this,b,qb(this)))});if(!arguments.length)return this.attr("class","");if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&&" "+pb(e)+" "){g=0;while(f=b[g++])while(d.indexOf(" "+f+" ")>-1)d=d.replace(" "+f+" "," ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},toggleClass:function(a,b){var c=typeof a;return"boolean"==typeof b&&"string"===c?b?this.addClass(a):this.removeClass(a):r.isFunction(a)?this.each(function(c){r(this).toggleClass(a.call(this,c,qb(this),b),b)}):this.each(function(){var b,d,e,f;if("string"===c){d=0,e=r(this),f=a.match(L)||[];while(b=f[d++])e.hasClass(b)?e.removeClass(b):e.addClass(b)}else void 0!==a&&"boolean"!==c||(b=qb(this),b&&W.set(this,"__className__",b),this.setAttribute&
 &this.setAttribute("class",b||a===!1?"":W.get(this,"__className__")||""))})},hasClass:function(a){var b,c,d=0;b=" "+a+" ";while(c=this[d++])if(1===c.nodeType&&(" "+pb(qb(c))+" ").indexOf(b)>-1)return!0;return!1}});var rb=/\r/g;r.fn.extend({val:function(a){var b,c,d,e=this[0];{if(arguments.length)return d=r.isFunction(a),this.each(function(c){var e;1===this.nodeType&&(e=d?a.call(this,c,r(this).val()):a,null==e?e="":"number"==typeof e?e+="":Array.isArray(e)&&(e=r.map(e,function(a){return null==a?"":a+""})),b=r.valHooks[this.type]||r.valHooks[this.nodeName.toLowerCase()],b&&"set"in b&&void 0!==b.set(this,e,"value")||(this.value=e))});if(e)return b=r.valHooks[e.type]||r.valHooks[e.nodeName.toLowerCase()],b&&"get"in b&&void 0!==(c=b.get(e,"value"))?c:(c=e.value,"string"==typeof c?c.replace(rb,""):null==c?"":c)}}}),r.extend({valHooks:{option:{get:function(a){var b=r.find.attr(a,"value");return null!=b?b:pb(r.text(a))}},select:{get:function(a){var b,c,d,e=a.options,f=a.selectedIndex,g="sel
 ect-one"===a.type,h=g?null:[],i=g?f+1:e.length;for(d=f<0?i:g?f:0;d<i;d++)if(c=e[d],(c.selected||d===f)&&!c.disabled&&(!c.parentNode.disabled||!B(c.parentNode,"optgroup"))){if(b=r(c).val(),g)return b;h.push(b)}return h},set:function(a,b){var c,d,e=a.options,f=r.makeArray(b),g=e.length;while(g--)d=e[g],(d.selected=r.inArray(r.valHooks.option.get(d),f)>-1)&&(c=!0);return c||(a.selectedIndex=-1),f}}}}),r.each(["radio","checkbox"],function(){r.valHooks[this]={set:function(a,b){if(Array.isArray(b))return a.checked=r.inArray(r(a).val(),b)>-1}},o.checkOn||(r.valHooks[this].get=function(a){return null===a.getAttribute("value")?"on":a.value})});var sb=/^(?:focusinfocus|focusoutblur)$/;r.extend(r.event,{trigger:function(b,c,e,f){var g,h,i,j,k,m,n,o=[e||d],p=l.call(b,"type")?b.type:b,q=l.call(b,"namespace")?b.namespace.split("."):[];if(h=i=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!sb.test(p+r.event.triggered)&&(p.indexOf(".")>-1&&(q=p.split("."),p=q.shift(),q.sort()),k=p.indexOf(":")<0&&"on"+p,b=
 b[r.expando]?b:new r.Event(p,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=q.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:r.makeArray(c,[b]),n=r.event.special[p]||{},f||!n.trigger||n.trigger.apply(e,c)!==!1)){if(!f&&!n.noBubble&&!r.isWindow(e)){for(j=n.delegateType||p,sb.test(j+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),i=h;i===(e.ownerDocument||d)&&o.push(i.defaultView||i.parentWindow||a)}g=0;while((h=o[g++])&&!b.isPropagationStopped())b.type=g>1?j:n.bindType||p,m=(W.get(h,"events")||{})[b.type]&&W.get(h,"handle"),m&&m.apply(h,c),m=k&&h[k],m&&m.apply&&U(h)&&(b.result=m.apply(h,c),b.result===!1&&b.preventDefault());return b.type=p,f||b.isDefaultPrevented()||n._default&&n._default.apply(o.pop(),c)!==!1||!U(e)||k&&r.isFunction(e[p])&&!r.isWindow(e)&&(i=e[k],i&&(e[k]=null),r.event.triggered=p,e[p](),r.event.triggered=void 0,i&&(e[k]=i)),b.result}},simulate:function(a,b,c)
 {var d=r.extend(new r.Event,c,{type:a,isSimulated:!0});r.event.trigger(d,null,b)}}),r.fn.extend({trigger:function(a,b){return this.each(function(){r.event.trigger(a,b,this)})},triggerHandler:function(a,b){var c=this[0];if(c)return r.event.trigger(a,b,c,!0)}}),r.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(a,b){r.fn[b]=function(a,c){return arguments.length>0?this.on(b,null,a,c):this.trigger(b)}}),r.fn.extend({hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),o.focusin="onfocusin"in a,o.focusin||r.each({focus:"focusin",blur:"focusout"},function(a,b){var c=function(a){r.event.simulate(b,a.target,r.event.fix(a))};r.event.special[b]={setup:function(){var d=this.ownerDocument||this,e=W.access(d,b);e||d.addEventListener(a,c,!0),W.access(d,b,(e||0)+1)},teardown:function(){var d=this.ownerDocument||this,e=W.access(
 d,b)-1;e?W.access(d,b,e):(d.removeEventListener(a,c,!0),W.remove(d,b))}}});var tb=a.location,ub=r.now(),vb=/\?/;r.parseXML=function(b){var c;if(!b||"string"!=typeof b)return null;try{c=(new a.DOMParser).parseFromString(b,"text/xml")}catch(d){c=void 0}return c&&!c.getElementsByTagName("parsererror").length||r.error("Invalid XML: "+b),c};var wb=/\[\]$/,xb=/\r?\n/g,yb=/^(?:submit|button|image|reset|file)$/i,zb=/^(?:input|select|textarea|keygen)/i;function Ab(a,b,c,d){var e;if(Array.isArray(b))r.each(b,function(b,e){c||wb.test(a)?d(a,e):Ab(a+"["+("object"==typeof e&&null!=e?b:"")+"]",e,c,d)});else if(c||"object"!==r.type(b))d(a,b);else for(e in b)Ab(a+"["+e+"]",b[e],c,d)}r.param=function(a,b){var c,d=[],e=function(a,b){var c=r.isFunction(b)?b():b;d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(null==c?"":c)};if(Array.isArray(a)||a.jquery&&!r.isPlainObject(a))r.each(a,function(){e(this.name,this.value)});else for(c in a)Ab(c,a[c],b,e);return d.join("&")},r.fn.extend({serialize:f
 unction(){return r.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var a=r.prop(this,"elements");return a?r.makeArray(a):this}).filter(function(){var a=this.type;return this.name&&!r(this).is(":disabled")&&zb.test(this.nodeName)&&!yb.test(a)&&(this.checked||!ja.test(a))}).map(function(a,b){var c=r(this).val();return null==c?null:Array.isArray(c)?r.map(c,function(a){return{name:b.name,value:a.replace(xb,"\r\n")}}):{name:b.name,value:c.replace(xb,"\r\n")}}).get()}});var Bb=/%20/g,Cb=/#.*$/,Db=/([?&])_=[^&]*/,Eb=/^(.*?):[ \t]*([^\r\n]*)$/gm,Fb=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Gb=/^(?:GET|HEAD)$/,Hb=/^\/\//,Ib={},Jb={},Kb="*/".concat("*"),Lb=d.createElement("a");Lb.href=tb.href;function Mb(a){return function(b,c){"string"!=typeof b&&(c=b,b="*");var d,e=0,f=b.toLowerCase().match(L)||[];if(r.isFunction(c))while(d=f[e++])"+"===d[0]?(d=d.slice(1)||"*",(a[d]=a[d]||[]).unshift(c)):(a[d]=a[d]||[]).push(c)}}function Nb(a,b,c,d){var e
 ={},f=a===Jb;function g(h){var i;return e[h]=!0,r.each(a[h]||[],function(a,h){var j=h(b,c,d);return"string"!=typeof j||f||e[j]?f?!(i=j):void 0:(b.dataTypes.unshift(j),g(j),!1)}),i}return g(b.dataTypes[0])||!e["*"]&&g("*")}function Ob(a,b){var c,d,e=r.ajaxSettings.flatOptions||{};for(c in b)void 0!==b[c]&&((e[c]?a:d||(d={}))[c]=b[c]);return d&&r.extend(!0,a,d),a}function Pb(a,b,c){var d,e,f,g,h=a.contents,i=a.dataTypes;while("*"===i[0])i.shift(),void 0===d&&(d=a.mimeType||b.getResponseHeader("Content-Type"));if(d)for(e in h)if(h[e]&&h[e].test(d)){i.unshift(e);break}if(i[0]in c)f=i[0];else{for(e in c){if(!i[0]||a.converters[e+" "+i[0]]){f=e;break}g||(g=e)}f=f||g}if(f)return f!==i[0]&&i.unshift(f),c[f]}function Qb(a,b,c,d){var e,f,g,h,i,j={},k=a.dataTypes.slice();if(k[1])for(g in a.converters)j[g.toLowerCase()]=a.converters[g];f=k.shift();while(f)if(a.responseFields[f]&&(c[a.responseFields[f]]=b),!i&&d&&a.dataFilter&&(b=a.dataFilter(b,a.dataType)),i=f,f=k.shift())if("*"===f)f=i;else if
 ("*"!==i&&i!==f){if(g=j[i+" "+f]||j["* "+f],!g)for(e in j)if(h=e.split(" "),h[1]===f&&(g=j[i+" "+h[0]]||j["* "+h[0]])){g===!0?g=j[e]:j[e]!==!0&&(f=h[0],k.unshift(h[1]));break}if(g!==!0)if(g&&a["throws"])b=g(b);else try{b=g(b)}catch(l){return{state:"parsererror",error:g?l:"No conversion from "+i+" to "+f}}}return{state:"success",data:b}}r.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:tb.href,type:"GET",isLocal:Fb.test(tb.protocol),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Kb,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/\bxml\b/,html:/\bhtml/,json:/\bjson\b/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":JSON.parse,"text xml":r.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(a,b){return b?Ob(Ob(a,r.ajaxSettings),b):Ob(r.ajaxSetting
 s,a)},ajaxPrefilter:Mb(Ib),ajaxTransport:Mb(Jb),ajax:function(b,c){"object"==typeof b&&(c=b,b=void 0),c=c||{};var e,f,g,h,i,j,k,l,m,n,o=r.ajaxSetup({},c),p=o.context||o,q=o.context&&(p.nodeType||p.jquery)?r(p):r.event,s=r.Deferred(),t=r.Callbacks("once memory"),u=o.statusCode||{},v={},w={},x="canceled",y={readyState:0,getResponseHeader:function(a){var b;if(k){if(!h){h={};while(b=Eb.exec(g))h[b[1].toLowerCase()]=b[2]}b=h[a.toLowerCase()]}return null==b?null:b},getAllResponseHeaders:function(){return k?g:null},setRequestHeader:function(a,b){return null==k&&(a=w[a.toLowerCase()]=w[a.toLowerCase()]||a,v[a]=b),this},overrideMimeType:function(a){return null==k&&(o.mimeType=a),this},statusCode:function(a){var b;if(a)if(k)y.always(a[y.status]);else for(b in a)u[b]=[u[b],a[b]];return this},abort:function(a){var b=a||x;return e&&e.abort(b),A(0,b),this}};if(s.promise(y),o.url=((b||o.url||tb.href)+"").replace(Hb,tb.protocol+"//"),o.type=c.method||c.type||o.method||o.type,o.dataTypes=(o.dataType
 ||"*").toLowerCase().match(L)||[""],null==o.crossDomain){j=d.createElement("a");try{j.href=o.url,j.href=j.href,o.crossDomain=Lb.protocol+"//"+Lb.host!=j.protocol+"//"+j.host}catch(z){o.crossDomain=!0}}if(o.data&&o.processData&&"string"!=typeof o.data&&(o.data=r.param(o.data,o.traditional)),Nb(Ib,o,c,y),k)return y;l=r.event&&o.global,l&&0===r.active++&&r.event.trigger("ajaxStart"),o.type=o.type.toUpperCase(),o.hasContent=!Gb.test(o.type),f=o.url.replace(Cb,""),o.hasContent?o.data&&o.processData&&0===(o.contentType||"").indexOf("application/x-www-form-urlencoded")&&(o.data=o.data.replace(Bb,"+")):(n=o.url.slice(f.length),o.data&&(f+=(vb.test(f)?"&":"?")+o.data,delete o.data),o.cache===!1&&(f=f.replace(Db,"$1"),n=(vb.test(f)?"&":"?")+"_="+ub++ +n),o.url=f+n),o.ifModified&&(r.lastModified[f]&&y.setRequestHeader("If-Modified-Since",r.lastModified[f]),r.etag[f]&&y.setRequestHeader("If-None-Match",r.etag[f])),(o.data&&o.hasContent&&o.contentType!==!1||c.contentType)&&y.setRequestHeader("Co
 ntent-Type",o.contentType),y.setRequestHeader("Accept",o.dataTypes[0]&&o.accepts[o.dataTypes[0]]?o.accepts[o.dataTypes[0]]+("*"!==o.dataTypes[0]?", "+Kb+"; q=0.01":""):o.accepts["*"]);for(m in o.headers)y.setRequestHeader(m,o.headers[m]);if(o.beforeSend&&(o.beforeSend.call(p,y,o)===!1||k))return y.abort();if(x="abort",t.add(o.complete),y.done(o.success),y.fail(o.error),e=Nb(Jb,o,c,y)){if(y.readyState=1,l&&q.trigger("ajaxSend",[y,o]),k)return y;o.async&&o.timeout>0&&(i=a.setTimeout(function(){y.abort("timeout")},o.timeout));try{k=!1,e.send(v,A)}catch(z){if(k)throw z;A(-1,z)}}else A(-1,"No Transport");function A(b,c,d,h){var j,m,n,v,w,x=c;k||(k=!0,i&&a.clearTimeout(i),e=void 0,g=h||"",y.readyState=b>0?4:0,j=b>=200&&b<300||304===b,d&&(v=Pb(o,y,d)),v=Qb(o,v,y,j),j?(o.ifModified&&(w=y.getResponseHeader("Last-Modified"),w&&(r.lastModified[f]=w),w=y.getResponseHeader("etag"),w&&(r.etag[f]=w)),204===b||"HEAD"===o.type?x="nocontent":304===b?x="notmodified":(x=v.state,m=v.data,n=v.error,j=!n)
 ):(n=x,!b&&x||(x="error",b<0&&(b=0))),y.status=b,y.statusText=(c||x)+"",j?s.resolveWith(p,[m,x,y]):s.rejectWith(p,[y,x,n]),y.statusCode(u),u=void 0,l&&q.trigger(j?"ajaxSuccess":"ajaxError",[y,o,j?m:n]),t.fireWith(p,[y,x]),l&&(q.trigger("ajaxComplete",[y,o]),--r.active||r.event.trigger("ajaxStop")))}return y},getJSON:function(a,b,c){return r.get(a,b,c,"json")},getScript:function(a,b){return r.get(a,void 0,b,"script")}}),r.each(["get","post"],function(a,b){r[b]=function(a,c,d,e){return r.isFunction(c)&&(e=e||d,d=c,c=void 0),r.ajax(r.extend({url:a,type:b,dataType:e,data:c,success:d},r.isPlainObject(a)&&a))}}),r._evalUrl=function(a){return r.ajax({url:a,type:"GET",dataType:"script",cache:!0,async:!1,global:!1,"throws":!0})},r.fn.extend({wrapAll:function(a){var b;return this[0]&&(r.isFunction(a)&&(a=a.call(this[0])),b=r(a,this[0].ownerDocument).eq(0).clone(!0),this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstElementChild)a=a.firstElementChild;return a}
 ).append(this)),this},wrapInner:function(a){return r.isFunction(a)?this.each(function(b){r(this).wrapInner(a.call(this,b))}):this.each(function(){var b=r(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=r.isFunction(a);return this.each(function(c){r(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(a){return this.parent(a).not("body").each(function(){r(this).replaceWith(this.childNodes)}),this}}),r.expr.pseudos.hidden=function(a){return!r.expr.pseudos.visible(a)},r.expr.pseudos.visible=function(a){return!!(a.offsetWidth||a.offsetHeight||a.getClientRects().length)},r.ajaxSettings.xhr=function(){try{return new a.XMLHttpRequest}catch(b){}};var Rb={0:200,1223:204},Sb=r.ajaxSettings.xhr();o.cors=!!Sb&&"withCredentials"in Sb,o.ajax=Sb=!!Sb,r.ajaxTransport(function(b){var c,d;if(o.cors||Sb&&!b.crossDomain)return{send:function(e,f){var g,h=b.xhr();if(h.open(b.type,b.url,b.async,b.username,b.password),b.xhrFields)for(g in b.xhrFields)h[g]=b.xhrFields[g];b.m
 imeType&&h.overrideMimeType&&h.overrideMimeType(b.mimeType),b.crossDomain||e["X-Requested-With"]||(e["X-Requested-With"]="XMLHttpRequest");for(g in e)h.setRequestHeader(g,e[g]);c=function(a){return function(){c&&(c=d=h.onload=h.onerror=h.onabort=h.onreadystatechange=null,"abort"===a?h.abort():"error"===a?"number"!=typeof h.status?f(0,"error"):f(h.status,h.statusText):f(Rb[h.status]||h.status,h.statusText,"text"!==(h.responseType||"text")||"string"!=typeof h.responseText?{binary:h.response}:{text:h.responseText},h.getAllResponseHeaders()))}},h.onload=c(),d=h.onerror=c("error"),void 0!==h.onabort?h.onabort=d:h.onreadystatechange=function(){4===h.readyState&&a.setTimeout(function(){c&&d()})},c=c("abort");try{h.send(b.hasContent&&b.data||null)}catch(i){if(c)throw i}},abort:function(){c&&c()}}}),r.ajaxPrefilter(function(a){a.crossDomain&&(a.contents.script=!1)}),r.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},conte
 nts:{script:/\b(?:java|ecma)script\b/},converters:{"text script":function(a){return r.globalEval(a),a}}}),r.ajaxPrefilter("script",function(a){void 0===a.cache&&(a.cache=!1),a.crossDomain&&(a.type="GET")}),r.ajaxTransport("script",function(a){if(a.crossDomain){var b,c;return{send:function(e,f){b=r("<script>").prop({charset:a.scriptCharset,src:a.url}).on("load error",c=function(a){b.remove(),c=null,a&&f("error"===a.type?404:200,a.type)}),d.head.appendChild(b[0])},abort:function(){c&&c()}}}});var Tb=[],Ub=/(=)\?(?=&|$)|\?\?/;r.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var a=Tb.pop()||r.expando+"_"+ub++;return this[a]=!0,a}}),r.ajaxPrefilter("json jsonp",function(b,c,d){var e,f,g,h=b.jsonp!==!1&&(Ub.test(b.url)?"url":"string"==typeof b.data&&0===(b.contentType||"").indexOf("application/x-www-form-urlencoded")&&Ub.test(b.data)&&"data");if(h||"jsonp"===b.dataTypes[0])return e=b.jsonpCallback=r.isFunction(b.jsonpCallback)?b.jsonpCallback():b.jsonpCallback,h?b[h]=b[h].replace(Ub
 ,"$1"+e):b.jsonp!==!1&&(b.url+=(vb.test(b.url)?"&":"?")+b.jsonp+"="+e),b.converters["script json"]=function(){return g||r.error(e+" was not called"),g[0]},b.dataTypes[0]="json",f=a[e],a[e]=function(){g=arguments},d.always(function(){void 0===f?r(a).removeProp(e):a[e]=f,b[e]&&(b.jsonpCallback=c.jsonpCallback,Tb.push(e)),g&&r.isFunction(f)&&f(g[0]),g=f=void 0}),"script"}),o.createHTMLDocument=function(){var a=d.implementation.createHTMLDocument("").body;return a.innerHTML="<form></form><form></form>",2===a.childNodes.length}(),r.parseHTML=function(a,b,c){if("string"!=typeof a)return[];"boolean"==typeof b&&(c=b,b=!1);var e,f,g;return b||(o.createHTMLDocument?(b=d.implementation.createHTMLDocument(""),e=b.createElement("base"),e.href=d.location.href,b.head.appendChild(e)):b=d),f=C.exec(a),g=!c&&[],f?[b.createElement(f[1])]:(f=qa([a],b,g),g&&g.length&&r(g).remove(),r.merge([],f.childNodes))},r.fn.load=function(a,b,c){var d,e,f,g=this,h=a.indexOf(" ");return h>-1&&(d=pb(a.slice(h)),a=a.sl
 ice(0,h)),r.isFunction(b)?(c=b,b=void 0):b&&"object"==typeof b&&(e="POST"),g.length>0&&r.ajax({url:a,type:e||"GET",dataType:"html",data:b}).done(function(a){f=arguments,g.html(d?r("<div>").append(r.parseHTML(a)).find(d):a)}).always(c&&function(a,b){g.each(function(){c.apply(this,f||[a.responseText,b,a])})}),this},r.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(a,b){r.fn[b]=function(a){return this.on(b,a)}}),r.expr.pseudos.animated=function(a){return r.grep(r.timers,function(b){return a===b.elem}).length},r.offset={setOffset:function(a,b,c){var d,e,f,g,h,i,j,k=r.css(a,"position"),l=r(a),m={};"static"===k&&(a.style.position="relative"),h=l.offset(),f=r.css(a,"top"),i=r.css(a,"left"),j=("absolute"===k||"fixed"===k)&&(f+i).indexOf("auto")>-1,j?(d=l.position(),g=d.top,e=d.left):(g=parseFloat(f)||0,e=parseFloat(i)||0),r.isFunction(b)&&(b=b.call(a,c,r.extend({},h))),null!=b.top&&(m.top=b.top-h.top+g),null!=b.left&&(m.left=b.left-h.left+e),"using
 "in b?b.using.call(a,m):l.css(m)}},r.fn.extend({offset:function(a){if(arguments.length)return void 0===a?this:this.each(function(b){r.offset.setOffset(this,a,b)});var b,c,d,e,f=this[0];if(f)return f.getClientRects().length?(d=f.getBoundingClientRect(),b=f.ownerDocument,c=b.documentElement,e=b.defaultView,{top:d.top+e.pageYOffset-c.clientTop,left:d.left+e.pageXOffset-c.clientLeft}):{top:0,left:0}},position:function(){if(this[0]){var a,b,c=this[0],d={top:0,left:0};return"fixed"===r.css(c,"position")?b=c.getBoundingClientRect():(a=this.offsetParent(),b=this.offset(),B(a[0],"html")||(d=a.offset()),d={top:d.top+r.css(a[0],"borderTopWidth",!0),left:d.left+r.css(a[0],"borderLeftWidth",!0)}),{top:b.top-d.top-r.css(c,"marginTop",!0),left:b.left-d.left-r.css(c,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var a=this.offsetParent;while(a&&"static"===r.css(a,"position"))a=a.offsetParent;return a||ra})}}),r.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function
 (a,b){var c="pageYOffset"===b;r.fn[a]=function(d){return T(this,function(a,d,e){var f;return r.isWindow(a)?f=a:9===a.nodeType&&(f=a.defaultView),void 0===e?f?f[b]:a[d]:void(f?f.scrollTo(c?f.pageXOffset:e,c?e:f.pageYOffset):a[d]=e)},a,d,arguments.length)}}),r.each(["top","left"],function(a,b){r.cssHooks[b]=Pa(o.pixelPosition,function(a,c){if(c)return c=Oa(a,b),Ma.test(c)?r(a).position()[b]+"px":c})}),r.each({Height:"height",Width:"width"},function(a,b){r.each({padding:"inner"+a,content:b,"":"outer"+a},function(c,d){r.fn[d]=function(e,f){var g=arguments.length&&(c||"boolean"!=typeof e),h=c||(e===!0||f===!0?"margin":"border");return T(this,function(b,c,e){var f;return r.isWindow(b)?0===d.indexOf("outer")?b["inner"+a]:b.document.documentElement["client"+a]:9===b.nodeType?(f=b.documentElement,Math.max(b.body["scroll"+a],f["scroll"+a],b.body["offset"+a],f["offset"+a],f["client"+a])):void 0===e?r.css(b,c,h):r.style(b,c,e,h)},b,g?e:void 0,g)}})}),r.fn.extend({bind:function(a,b,c){return thi
 s.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return 1===arguments.length?this.off(a,"**"):this.off(b,a||"**",c)}}),r.holdReady=function(a){a?r.readyWait++:r.ready(!0)},r.isArray=Array.isArray,r.parseJSON=JSON.parse,r.nodeName=B,"function"==typeof define&&define.amd&&define("jquery",[],function(){return r});var Vb=a.jQuery,Wb=a.$;return r.noConflict=function(b){return a.$===r&&(a.$=Wb),b&&a.jQuery===r&&(a.jQuery=Vb),r},b||(a.jQuery=a.$=r),r});