[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99013960
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+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;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * batches. Several limits apply to the maximum "width" of this merge. For
+ * example, we each open spill run consumes a file handle, and we may wish
+ * to limit the number of file handles. A consolidation phase combines
+ * in-m

[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99017885
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+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;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * batches. Several limits apply to the maximum "width" of this merge. For
+ * example, we each open spill run consumes a file handle, and we may wish
+ * to limit the number of file handles. A consolidation phase combines
+ * in-m

[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99039654
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
 ---
@@ -0,0 +1,271 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.exec.compile.sig.GeneratorMapping;
+import org.apache.drill.exec.compile.sig.MappingSet;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.vector.CopyUtil;
+
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+
+/**
+ * Generates and manages the data-specific classes for this operator.
+ * 
+ * Several of the code generation methods take a batch, but the methods
+ * are called for many batches, and generate code only for the first one.
+ * Better would be to generate code from a schema; but Drill is not set
+ * up for that at present.
+ */
+
+public class OperatorCodeGenerator {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(OperatorCodeGenerator.class);
+
+  protected static final MappingSet MAIN_MAPPING = new MappingSet((String) 
null, null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+  protected static final MappingSet LEFT_MAPPING = new 
MappingSet("leftIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+  protected static final MappingSet RIGHT_MAPPING = new 
MappingSet("rightIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+
+  private static final GeneratorMapping COPIER_MAPPING = new 
GeneratorMapping("doSetup", "doCopy", null, null);
+  private static final MappingSet COPIER_MAPPING_SET = new 
MappingSet(COPIER_MAPPING, COPIER_MAPPING);
+
+  private final FragmentContext context;
+  @SuppressWarnings("unused")
+  private BatchSchema schema;
+
+  /**
+   * A single PriorityQueueCopier instance is used for 2 purposes:
+   * 1. Merge sorted batches before spilling
+   * 2. Merge sorted batches when all incoming data fits in memory
+   */
+
+  private PriorityQueueCopier copier;
+  private final Sort popConfig;
+  private MSorter mSorter;
+
+  /**
+   * Generated sort operation used to sort each incoming batch according to
+   * the sort criteria specified in the {@link ExternalSort} definition of
+   * this operator.
+   */
+
+  private SingleBatchSorter sorter;
+
+  public OperatorCodeGenerator(FragmentContext context, Sort popConfig) {
+this.context = context;
+this.popConfig = popConfig;
+  }
+
+  public void setSchema(BatchSchema schema) {
+close();
+this.schema = schema;
+  }
+
+  public void

[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99028564
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
--- End diff --

Can the "builder" and the "mSorter" become internal variables of sort() 
below, and then cleared/closed before sort() returns ?  And then close() would 
be a no-op.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99041705
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpillSet.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
+ *
+ * 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.physical.impl.xsort.managed;
--- End diff --

This class should be in a more general package, to be used also for other 
operators that spill (future work).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99024407
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+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;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * batches. Several limits apply to the maximum "width" of this merge. For
+ * example, we each open spill run consumes a file handle, and we may wish
+ * to limit the number of file handles. A consolidation phase combines
+ * in-m

[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99028635
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
+  private final FragmentContext context;
+  private final BufferAllocator oAllocator;
+  private SelectionVector4 sv4;
+  private final OperatorCodeGenerator opCg;
+  private int batchCount;
+
+  public InMemorySorter(FragmentContext context, BufferAllocator 
allocator, OperatorCodeGenerator opCg) {
+this.context = context;
+this.oAllocator = allocator;
+this.opCg = opCg;
+  }
+
+  public SelectionVector4 sort(LinkedList 
batchGroups, VectorAccessible batch,
--- End diff --

maybe add some Javadoc in this file ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99045403
  
--- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
@@ -177,13 +177,47 @@ drill.exec: {
   sort: {
 purge.threshold : 1000,
 external: {
-  batch.size : 4000,
+  // 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.
+  // 0 = unlimited
+  batch_limit: 0
+  // Limit on the amount of memory used for xsort. Overrides the
+  // value provided by Foreman. Primarily for testing.
+  // 0 = unlimited, Supports HOCON memory suffixes.
+  mem_limit: 0
+  // Limit on the number of spilled batches that can be merged in
+  // a single pass. Limits the number of open file handles.
+  // 0 = unlimited
+  merge_limit: 0
   spill: {
-batch.size : 4000,
-group.size : 4,
-threshold : 4,
-directories : [ "/tmp/drill/spill" ],
-fs : "file:///"
+// Deprecated for managed xsort; used only by legacy xsort
+group.size: 4,
+// Deprecated for managed xsort; used only by legacy xsort
+threshold: 4,
+// Minimum number of in-memory batches to spill per spill file
+// Affects only spilling from memory to disk.
+// Primarily for testing.
+min_batches: 2,
+// Maximum number of in-memory batches to spill per spill file
+// Affects only spilling from memory to disk.
+// Primarily for testing.
+// 0 = unlimited
+max_batches: 0,
+// File system to use. Local file system by default.
+fs: "file:///"
+// List of directories to use. Directories are created
--- End diff --

List of (comma separated) directories


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99040449
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
 ---
@@ -0,0 +1,43 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.VectorAccessible;
+
+public interface PriorityQueueCopier extends AutoCloseable {
+  public static final long INITIAL_ALLOCATION = 1000;
+  public static final long MAX_ALLOCATION = 2000;
--- End diff --

Suggestion: change to  10_000_000L and 20_000_000L 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99027623
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
+  private final FragmentContext context;
+  private final BufferAllocator oAllocator;
+  private SelectionVector4 sv4;
+  private final OperatorCodeGenerator opCg;
+  private int batchCount;
+
+  public InMemorySorter(FragmentContext context, BufferAllocator 
allocator, OperatorCodeGenerator opCg) {
+this.context = context;
+this.oAllocator = allocator;
+this.opCg = opCg;
+  }
+
+  public SelectionVector4 sort(LinkedList 
batchGroups, VectorAccessible batch,
+VectorContainer destContainer) {
+if (builder != null) {
+  builder.clear();
+  builder.close();
+}
+builder = new SortRecordBatchBuilder(oAllocator);
+
+for (BatchGroup.InputBatch group : batchGroups) {
+  RecordBatchData rbd = new RecordBatchData(group.getContainer(), 
oAllocator);
+  rbd.setSv2(group.getSv2());
+  builder.add(rbd);
+}
+batchGroups.clear();
+
+try {
+  builder.build(context, destContainer);
+  sv4 = builder.getSv4();
+  mSorter = opCg.createNewMSorter(batch);
+  mSorter.setup(context, oAllocator, sv4, destContainer, 
sv4.getCount());
+} catch (SchemaChangeException e) {
+  throw UserException.unsupportedError(e)
+.message("Unexpected schema change - likely code error.")
+.build(logger);
+}
+
+// For testing memory-leaks, inject exception after mSorter finishes 
setup
+
ExternalSortBatch.injector.injectUnchecked(context.getExecutionControls(), 
ExternalSortBatch.INTERRUPTION_AFTER_SETUP);
+mSorter.sort(destContainer);
+
+// sort may have prematurely exited due to should continue returning 
false.
+if (!context.shouldContinue()) {
+  return null;
+}
+
+// For testing memory-leak purpose, inject exception after mSorter 
finishes sorting
+
ExternalSortBatch.injector.injectUnchecked(context.getExecutionControls(), 
ExternalSortBatch.INTERRUPTION_AFTER_SORT);
+sv4 = mSorter.getSV4();
+
+destContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
+return sv4;
+  }
+
+  @Override
+  public boolean next() {
+boolean more = sv4.next();
+if (more) { batchCount++; }
+return more;
+  }
+
+  @Override
+  public void close() {
+if (builder != null) {
+  builder.clear();
+  builder.close();
+}
+if (mSorter != null) {
+  mSorter.clear();
+}
+  }
+
+  @Override
+  public int getBatchCount() {
+   

[GitHub] drill pull request #717: DRILL-5080: Memory-managed version of external sort

2017-02-01 Thread Ben-Zvi
Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99036541
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
 ---
@@ -0,0 +1,237 @@
+/*
+ * 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.physical.impl.xsort.managed;
+
+import java.util.Queue;
+
+import javax.inject.Named;
+
+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;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.hadoop.util.IndexedSortable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+
+import io.netty.buffer.DrillBuf;
+
+public abstract class MSortTemplate implements MSorter, IndexedSortable {
+//  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(MSortTemplate.class);
+
+  private SelectionVector4 vector4;
+  private SelectionVector4 aux;
+  @SuppressWarnings("unused")
+  private long compares;
+
+  /**
+   * Holds offsets into the SV4 of the start of each batch
+   * (sorted run.)
+   */
+
+  private Queue runStarts = Queues.newLinkedBlockingQueue();
+  private FragmentContext context;
+
+  /**
+   * Controls the maximum size of batches exposed to downstream
+   */
+  private int desiredRecordBatchCount;
+
+  @Override
+  public void setup(final FragmentContext context, final BufferAllocator 
allocator, final SelectionVector4 vector4,
+final VectorContainer hyperBatch, int outputBatchSize) 
throws SchemaChangeException{
+// we pass in the local hyperBatch since that is where we'll be 
reading data.
+Preconditions.checkNotNull(vector4);
+this.vector4 = vector4.createNewWrapperCurrent();
+this.context = context;
+vector4.clear();
+doSetup(context, hyperBatch, null);
+
+// Populate the queue with the offset in the SV4 of each
+// batch. Note that this is expensive as it requires a scan
+// of all items to be sorted: potentially millions.
+
+runStarts.add(0);
+int batch = 0;
+final int totalCount = this.vector4.getTotalCount();
+for (int i = 0; i < totalCount; i++) {
+  final int newBatch = this.vector4.get(i) >>> 16;
+  if (newBatch == batch) {
+continue;
+  } else if (newBatch == batch + 1) {
+runStarts.add(i);
+batch = newBatch;
+  } else {
+throw new UnsupportedOperationException(String.format("Missing 
batch. batch: %d newBatch: %d", batch, newBatch));
+  }
+}
+
+// Create a temporary SV4 to hold the merged results.
+
+@SuppressWarnings("resource")
+final DrillBuf drillBuf = allocator.buffer(4 * totalCount);
+desiredRecordBatchCount = Math.min(outputBatchSize, 
Character.MAX_VALUE);
+desiredRecordBatchCount = Math.min(desiredRecordBatchCount, 
totalCount);
+aux = new SelectionVector4(drillBuf, totalCount, 
desiredRecordBatchCount);
+  }
+
+  /**
+   * For given recordCount how much memory does MSorter needs for its own 
purpose. This is used in
+   * ExternalSortBatch to make decisions about whether to spill or not.
+   *
+   * @param recordCount
+   * @return
+   */
+  public static long memoryNeeded(final int recordCount) {
+// We need 4 bytes (SV4) for each record.
+// The memory allocator will round this to the next
+// power of 2.
+
+return BaseAllocator.nextPowerOfTwo(recordCount * 4);
+  }
+
+  /**
+   * Given two regions within the 

NPE On UDF in Directory Query

2017-02-01 Thread Charles Givre
Hello everyone, 
I’m working on a UDF (https://github.com/cgivre/drill-geoip-functions 
) that does a bunch of IPGEO 
stuff.  So far it works relatively well, but I encountered a Null Pointer 
Exception that I’m not quite sure how to resolve. 

If I execute the query:

SELECT getCountryInfo( ip ) 
FROM dfs.`data.csv`

It works.  If I try:

SELECT getCountryInfo (ip)
FROM dfs.`data.*`

It works..
But, I have data that is spread across multiple directories and when I try to 
use this function like this, I get Null Pointer Exceptions.  Any suggestions?

SELECT getCountryInfo( ip )
FROM dfs.`dir1/`

(assuming dir1 contains other directories).

Thanks
— Charles

[jira] [Resolved] (DRILL-4179) Update UDF documentation now that classpath scanning is more strict

2017-02-01 Thread Julien Le Dem (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-4179?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Julien Le Dem resolved DRILL-4179.
--
Resolution: Fixed

> Update UDF documentation now that classpath scanning is more strict
> ---
>
> Key: DRILL-4179
> URL: https://issues.apache.org/jira/browse/DRILL-4179
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Documentation
>Reporter: Jason Altekruse
>Assignee: Julien Le Dem
>
> A few issues have come up with users that have UDFs that could be found with 
> 1.0-1.2, but fail to be loaded with 1.3. There were changes in 1.3 to speed 
> up finding all UDFs on the classpath made the setup a little more strict.
> Some discussions on the topic:
> DRILL-4178
> http://search-hadoop.com/m/qRVAXvthcn1xIHUm/+add+your+package+to+drill.classpath.scanning&subj=Re+UDFs+and+1+3



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


Re: OpenTSDB plugin development for Drill

2017-02-01 Thread Ted Dunning
Moving user@drill to bcc.

The key questions here have to do with the following:

1) how is the best way to write rules that will help drill force the query
into recognizable form

2) is there a good way to pass parameters to the data source. Things like
resampling parameters and resampling aggregator have no good corollary in
SQL so putting them into the table specification seems reasonable. But how
does that work? Are there examples?

3) assuming that the query can be massaged via optimizer rules or the users
can be trained, what is the best way to pick off parts of the query for
inclusion?  For instance, all data points in TSDB are tagged with keys and
values. That would make it seem like this query:

select avg(t.value), max(t.value)
from table(`tsdb/memory`, sample='1m', aggregate='avg') t
where t.tags.cluster = 'c1'
group by t.tags.host;

would be nice. The variables cluster and host would refer to tags while
value would refer to the value itself. How would these be accessed inside
the data source for push-down into TSDB? Are there examples?

Also, how would the parameters of the table function be accessed? How does
Drill know that the TSDB data source is to be used?







On Wed, Feb 1, 2017 at 2:01 AM, Dmintriy Gavrilovich <
dhavrilov...@cybervisiontech.com> wrote:

> Hi everyone.
>
> TLTR;
>
> I have started to develop an OpenTSDB Plugin for Drill available here:
> https://github.com/mapr-demos/drill/tree/openTSDB-plugin/
> contrib/storage-opentsdb
>
> This is a work in progress and I have some ideas, and questions, see below
>
> DETAILS
>
>
> I am developing a storage plugin for OpenTSDB time series DB and I faced
> some problems due to completely  different APIs that drill expect and TSDB
> uses.
>
> As OpenTSDB do not have any java client or jdbc driver, only REST API.
> Here is a sample json call to tsdb:
> {
> "start": 1356998400,
> "end": 1356998460,
> "queries": [
> {
> "aggregator": "sum",
> "metric": "sys.cpu.0",
> "rate": "true",
> "tags": {
> "host": "*",
> "dc": "lga"
> }
> },
> {
> "aggregator": "sum",
> "tsuids": [
> "010242",
> "010243"
>   ]
> }
> }
> ]
> }
>
> Sample query with filters:
> {
> "start": 1356998400,
> "end": 1356998460,
> "queries": [
> {
> "aggregator": "sum",
> "metric": "sys.cpu.0",
> "rate": "true",
> "filters": [
> {
>"type":"wildcard",
>"tagk":"host",
>"filter":"*",
>"groupBy":true
> },
> {
>"type":"literal_or",
>"tagk":"dc",
>"filter":"lga|lga1|lga2",
>"groupBy":false
> },
> ]
> },
> {
> "aggregator": "sum",
> "tsuids": [
> "010242",
> "010243"
>   ]
> }
> }
> ]
> }
>
> Sample response:
> [
> {
> "metric": "tsd.hbase.puts",
> "tags": {},
> "aggregatedTags": [
> "host"
> ],
> "annotations": [
> {
> "tsuid": "1CFBFB",
> "description": "Testing Annotations",
> "notes": "These would be details about the event, the
> description is just a summary",
> "custom": {
> "owner": "jdoe",
> "dept": "ops"
> },
> "endTime": 0,
> "startTime": 1365966062
> }
> ],
> "globalAnnotations": [
> {
> "description": "Notice",
> "notes": "DAL was down during this period",
> "custom": null,
> "endTime": 1365966164,
> "startTime": 1365966064
> }
> ],
> "tsuids": [
> "0023E302080601"
> ],
> "dps": {
> "1365966001": 25595461080,
> "1365966061": 25595542522,
> "1365966062": 25595543979,
> ...
> "1365973801": 25717417859
> }
> }
> ]
>
> So the main problem is to convert values from SQL syntax to OpenTSDB
> values and push it to the API. Also we do not have fixed columns. We have a
> map in our tag column and each tag can be a search filter. This cause
> problems then we try to perform search using where clause.
>
> Query string like where host = * and dc = lga should be transformed like
> this:
> "tags": {
> "host": "*",
> "dc": "lga"
> }
>
> I have already a working prototype available her

[GitHub] drill issue #735: DRILL-5237: FlattenRecordBatch loses nested fields from th...

2017-02-01 Thread amansinha100
Github user amansinha100 commented on the issue:

https://github.com/apache/drill/pull/735
  
lgtm.  +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #725: DRILL-5215: CTTAS: disallow temp tables in view expansion ...

2017-02-01 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/725
  
+1

LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


OpenTSDB plugin development for Drill

2017-02-01 Thread Dmintriy Gavrilovich
Hi everyone. 

TLTR;

I have started to develop an OpenTSDB Plugin for Drill available here:
https://github.com/mapr-demos/drill/tree/openTSDB-plugin/contrib/storage-opentsdb

This is a work in progress and I have some ideas, and questions, see below

DETAILS


I am developing a storage plugin for OpenTSDB time series DB and I faced some 
problems due to completely  different APIs that drill expect and TSDB uses. 

As OpenTSDB do not have any java client or jdbc driver, only REST API. 
Here is a sample json call to tsdb:
{
"start": 1356998400,
"end": 1356998460,
"queries": [
{
"aggregator": "sum",
"metric": "sys.cpu.0",
"rate": "true",
"tags": {
"host": "*",
"dc": "lga"
}
},
{
"aggregator": "sum",
"tsuids": [
"010242",
"010243"
  ]
}
}
]
}

Sample query with filters:
{
"start": 1356998400,
"end": 1356998460,
"queries": [
{
"aggregator": "sum",
"metric": "sys.cpu.0",
"rate": "true",
"filters": [
{
   "type":"wildcard",
   "tagk":"host",
   "filter":"*",
   "groupBy":true
},
{
   "type":"literal_or",
   "tagk":"dc",
   "filter":"lga|lga1|lga2",
   "groupBy":false
},
]
},
{
"aggregator": "sum",
"tsuids": [
"010242",
"010243"
  ]
}
}
]
}

Sample response: 
[
{
"metric": "tsd.hbase.puts",
"tags": {},
"aggregatedTags": [
"host"
],
"annotations": [
{
"tsuid": "1CFBFB",
"description": "Testing Annotations",
"notes": "These would be details about the event, the 
description is just a summary",
"custom": {
"owner": "jdoe",
"dept": "ops"
},
"endTime": 0,
"startTime": 1365966062
}
],
"globalAnnotations": [
{
"description": "Notice",
"notes": "DAL was down during this period",
"custom": null,
"endTime": 1365966164,
"startTime": 1365966064
}
],
"tsuids": [
"0023E302080601"
],
"dps": {
"1365966001": 25595461080,
"1365966061": 25595542522,
"1365966062": 25595543979,
...
"1365973801": 25717417859
}
}
]

So the main problem is to convert values from SQL syntax to OpenTSDB values and 
push it to the API. Also we do not have fixed columns. We have a map in our tag 
column and each tag can be a search filter. This cause problems then we try to 
perform search using where clause.

Query string like where host = * and dc = lga should be transformed like this: 
"tags": {
"host": "*",
"dc": "lga"
}

I have already a working prototype available here:
https://github.com/mapr-demos/drill/tree/openTSDB-plugin/contrib/storage-opentsdb

With the following supported SQL statement:

SELECT * FROM ;


Now I  would like to go further and implement more time series related features 
for example:

1- select avg|sum|min|max(speedmetric.value)
2- from openTSDB(metric=sensor.speed, downsample='1m', interpolate='avg') 
speedmetric 
3- where speedmetric.tags.id in (001, 002) 
4- and speedmetric.timestamp >='value' and speedmetric.timestamp <= 'value' 
5- group by speedmetric.tags.hostname


Where:

1 - Where the aggregation function, should be pushed down to the OpenTSDB REST 
Call
-> How can I override the aggregation function for my plugin

2 - I currently working on converting string from this clause to map to use it 
in TSDB query
3 - tags what we are searching for
4 - time period for search. In fact is is two timestamp values “from” and “to”. 
This values are required
5 - don’t exactly know how transform this to the TSDB API.   


Now we are using this syntax to use aggregation function :

The syntax for SELECT query with aggregation function is:
SELECT * FROM ;

It transforms it such api request: 
`/api/query?start=5y-ago&m=sum:warp.speed` as get request. More complicated 
requests should use post requests.

Many thanks, Dmitriy Gavrilovich
dhavrilov...@cybervisiontech.com

[GitHub] drill issue #685: Drill 5043: Function that returns a unique id per session/...

2017-02-01 Thread arina-ielchiieva
Github user arina-ielchiieva commented on the issue:

https://github.com/apache/drill/pull/685
  
@nagarajanchinnasamy thank you, looks good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #735: DRILL-5237: Same query produces different/wrong res...

2017-02-01 Thread Serhii-Harnyk
GitHub user Serhii-Harnyk opened a pull request:

https://github.com/apache/drill/pull/735

DRILL-5237: Same query produces different/wrong results

Fixed loss of nested fields from the schema when the first 
FlattenRecordBatch is empty

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/Serhii-Harnyk/drill DRILL-5237

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/735.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #735


commit 800953b7ec924ea31ffc21327950c84528f3c492
Author: Serhii-Harnyk 
Date:   2017-01-27T15:36:10Z

DRILL-5237: Same query produces different/wrong results




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Created] (DRILL-5237) Same query produces different/wrong results

2017-02-01 Thread Serhii Harnyk (JIRA)
Serhii Harnyk created DRILL-5237:


 Summary: Same query produces different/wrong results
 Key: DRILL-5237
 URL: https://issues.apache.org/jira/browse/DRILL-5237
 Project: Apache Drill
  Issue Type: Bug
Reporter: Serhii Harnyk
Assignee: Serhii Harnyk


Query 
{code:sql}
select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as b 
from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 'a1');
{code}
returns different results for different values for option 
planner.width.max_per_node
With options 
{code:sql}
alter session set `planner.slice_target` = 1;
alter session set `planner.width.max_per_node` = 1;
{code}
query returns correct result:
{noformat}
+--+
| col  |
+--+
| 3|
+--+
{noformat}
but with options 
{code:sql}
alter session set `planner.slice_target` = 1;
alter session set `planner.width.max_per_node` = 3;
{code}
the same query returns wrong result 
{noformat}
+--+
| col  |
+--+
| 2|
+--+
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)