Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 50D34200D2B for ; Thu, 2 Nov 2017 16:28:20 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 4F480160BDE; Thu, 2 Nov 2017 15:28:20 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 6E47E1609EE for ; Thu, 2 Nov 2017 16:28:19 +0100 (CET) Received: (qmail 24225 invoked by uid 500); 2 Nov 2017 15:28:18 -0000 Mailing-List: contact dev-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@drill.apache.org Delivered-To: mailing list dev@drill.apache.org Received: (qmail 24214 invoked by uid 99); 2 Nov 2017 15:28:18 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 02 Nov 2017 15:28:18 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 32A59DFFB3; Thu, 2 Nov 2017 15:28:18 +0000 (UTC) From: Vlad-Storona To: dev@drill.apache.org Reply-To: dev@drill.apache.org References: In-Reply-To: Subject: [GitHub] drill pull request #774: DRILL-5337: OpenTSDB storage plugin Content-Type: text/plain Message-Id: <20171102152818.32A59DFFB3@git1-us-west.apache.org> Date: Thu, 2 Nov 2017 15:28:18 +0000 (UTC) archived-at: Thu, 02 Nov 2017 15:28:20 -0000 Github user Vlad-Storona commented on a diff in the pull request: https://github.com/apache/drill/pull/774#discussion_r148567870 --- Diff: contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBRecordReader.java --- @@ -0,0 +1,263 @@ +/* + * 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.openTSDB; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.exceptions.UserException; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.logical.ValidationError; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.common.types.TypeProtos.MajorType; +import org.apache.drill.common.types.TypeProtos.MinorType; +import org.apache.drill.common.types.Types; +import org.apache.drill.exec.exception.SchemaChangeException; +import org.apache.drill.exec.expr.TypeHelper; +import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.physical.impl.OutputMutator; +import org.apache.drill.exec.record.MaterializedField; +import org.apache.drill.exec.store.AbstractRecordReader; +import org.apache.drill.exec.store.openTSDB.client.OpenTSDBTypes; +import org.apache.drill.exec.store.openTSDB.client.Schema; +import org.apache.drill.exec.store.openTSDB.client.Service; +import org.apache.drill.exec.store.openTSDB.dto.ColumnDTO; +import org.apache.drill.exec.store.openTSDB.dto.MetricDTO; +import org.apache.drill.exec.vector.NullableFloat8Vector; +import org.apache.drill.exec.vector.NullableTimeStampVector; +import org.apache.drill.exec.vector.NullableVarCharVector; +import org.apache.drill.exec.vector.ValueVector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.nio.charset.StandardCharsets.UTF_8; + +public class OpenTSDBRecordReader extends AbstractRecordReader { + + private static final Logger log = LoggerFactory.getLogger(OpenTSDBRecordReader.class); + + private static final Map TYPES; + + private Service db; + + private Iterator tableIterator; + private OutputMutator output; + private ImmutableList projectedCols; + private OpenTSDBSubScan.OpenTSDBSubScanSpec subScanSpec; + + OpenTSDBRecordReader(Service client, OpenTSDBSubScan.OpenTSDBSubScanSpec subScanSpec, + List projectedColumns) throws IOException { + setColumns(projectedColumns); + this.db = client; + this.subScanSpec = subScanSpec; + db.setupQueryParameters(subScanSpec.getTableName()); + log.debug("Scan spec: {}", subScanSpec); + } + + @Override + public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { + this.output = output; + Set tables = db.getTablesFromDB(); + if (tables == null || tables.isEmpty()) { + throw new ValidationError(String.format("Table '%s' not found or it's empty", subScanSpec.getTableName())); + } + this.tableIterator = tables.iterator(); + } + + @Override + public int next() { + try { + return processOpenTSDBTablesData(); + } catch (SchemaChangeException e) { + log.info(e.toString()); + return 0; + } + } + + @Override + protected boolean isSkipQuery() { + return super.isSkipQuery(); + } + + @Override + public void close() throws Exception { + } + + static { + TYPES = ImmutableMap.builder() + .put(OpenTSDBTypes.STRING, MinorType.VARCHAR) + .put(OpenTSDBTypes.DOUBLE, MinorType.FLOAT8) + .put(OpenTSDBTypes.TIMESTAMP, MinorType.TIMESTAMP) + .build(); + } + + private static class ProjectedColumnInfo { + ValueVector vv; + ColumnDTO openTSDBColumn; + } + + private int processOpenTSDBTablesData() throws SchemaChangeException { + int rowCounter = 0; + while (tableIterator.hasNext()) { + MetricDTO metricDTO = tableIterator.next(); + rowCounter = addRowResult(metricDTO, rowCounter); + } + return rowCounter; + } + + private int addRowResult(MetricDTO table, int rowCounter) throws SchemaChangeException { + setupProjectedColsIfItNull(); + for (String time : table.getDps().keySet()) { + String value = table.getDps().get(time); + setupDataToDrillTable(table, time, value, table.getTags(), rowCounter); + rowCounter++; + } + return rowCounter; + } + + private void setupProjectedColsIfItNull() throws SchemaChangeException { + if (projectedCols == null) { + initCols(new Schema(db, subScanSpec.getTableName())); + } + } + + private void setupDataToDrillTable(MetricDTO table, String timestamp, String value, Map tags, int rowCount) { + for (ProjectedColumnInfo pci : projectedCols) { + switch (pci.openTSDBColumn.getColumnName()) { + case "metric": + setStringColumnValue(table.getMetric(), pci, rowCount); + break; + case "aggregate tags": + setStringColumnValue(table.getAggregateTags().toString(), pci, rowCount); + break; + case "timestamp": + setTimestampColumnValue(timestamp, pci, rowCount); + break; + case "aggregated value": + setDoubleColumnValue(value, pci, rowCount); + break; + default: + setStringColumnValue(tags.get(pci.openTSDBColumn.getColumnName()), pci, rowCount); + } + } + } + + private void setTimestampColumnValue(String timestamp, ProjectedColumnInfo pci, int rowCount) { + setTimestampColumnValue(timestamp != null ? Long.parseLong(timestamp) : Long.parseLong("0"), pci, rowCount); --- End diff -- I will change it. ---