├── .gitignore ├── src ├── main │ └── java │ │ ├── ml │ │ └── littlebulb │ │ │ └── presto │ │ │ └── kudu │ │ │ ├── properties │ │ │ ├── RangePartitionDefinition.java │ │ │ ├── RangePartition.java │ │ │ ├── HashPartitionDefinition.java │ │ │ ├── PartitionDesign.java │ │ │ ├── RangeBoundValue.java │ │ │ ├── ColumnDesign.java │ │ │ ├── RangeBoundValueDeserializer.java │ │ │ ├── RangeBoundValueSerializer.java │ │ │ └── KuduTableProperties.java │ │ │ ├── KuduTransactionHandle.java │ │ │ ├── KuduPlugin.java │ │ │ ├── SchemaAlreadyExistsException.java │ │ │ ├── Types.java │ │ │ ├── KuduConnectorId.java │ │ │ ├── KuduRecordCursorWithVirtualRowId.java │ │ │ ├── KuduInsertTableHandle.java │ │ │ ├── KuduClientSession.java │ │ │ ├── KuduPageSinkProvider.java │ │ │ ├── KuduExtendedTableHandle.java │ │ │ ├── KuduPageSourceProvider.java │ │ │ ├── KuduSplitManager.java │ │ │ ├── KuduRecordSetProvider.java │ │ │ ├── KuduHandleResolver.java │ │ │ ├── KuduConnectorFactory.java │ │ │ ├── KuduSplit.java │ │ │ ├── KuduOutputTableHandle.java │ │ │ ├── procedures │ │ │ └── RangePartitionProcedures.java │ │ │ ├── KuduUpdatablePageSource.java │ │ │ ├── KuduTableHandle.java │ │ │ ├── RowHelper.java │ │ │ ├── KuduTableLayoutHandle.java │ │ │ ├── KuduRecordSet.java │ │ │ ├── KuduColumnHandle.java │ │ │ ├── KuduClientConfig.java │ │ │ ├── KuduModule.java │ │ │ ├── KuduConnector.java │ │ │ ├── KuduRecordCursor.java │ │ │ ├── KuduPageSink.java │ │ │ ├── TypeHelper.java │ │ │ └── KuduMetadata.java │ │ └── org │ │ └── apache │ │ └── kudu │ │ └── client │ │ └── KeyEncoderAccessor.java └── test │ └── java │ └── ml │ └── littlebulb │ └── presto │ └── kudu │ ├── properties │ └── RangePartitionSerializationTest.java │ ├── TestKuduPlugin.java │ ├── TestSchemaNoExist.java │ ├── TestKuduIntegrationSmokeTest.java │ ├── TestDecimalColumnsTest.java │ ├── TestIntegerColumnsTest.java │ ├── KuduQueryRunnerFactory.java │ └── TestRangePartitioningTest.java ├── pom.xml ├── LICENSE └── README.md /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # BlueJ files 8 | *.ctxt 9 | 10 | # Mobile Tools for Java (J2ME) 11 | .mtj.tmp/ 12 | 13 | # Package Files # 14 | *.jar 15 | *.war 16 | *.ear 17 | *.zip 18 | *.tar.gz 19 | *.rar 20 | 21 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 22 | hs_err_pid* 23 | /.idea 24 | /target 25 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/RangePartitionDefinition.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import java.util.List; 4 | 5 | public class RangePartitionDefinition { 6 | private List columns; 7 | 8 | public List getColumns() { 9 | return columns; 10 | } 11 | 12 | public void setColumns(List columns) { 13 | this.columns = columns; 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/RangePartition.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | public class RangePartition { 4 | private RangeBoundValue lower; 5 | private RangeBoundValue upper; 6 | 7 | public RangeBoundValue getLower() { 8 | return lower; 9 | } 10 | 11 | public void setLower(RangeBoundValue lower) { 12 | this.lower = lower; 13 | } 14 | 15 | public RangeBoundValue getUpper() { 16 | return upper; 17 | } 18 | 19 | public void setUpper(RangeBoundValue upper) { 20 | this.upper = upper; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/HashPartitionDefinition.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import java.util.List; 4 | 5 | public class HashPartitionDefinition { 6 | private List columns; 7 | private int buckets; 8 | 9 | public List getColumns() { 10 | return columns; 11 | } 12 | 13 | public void setColumns(List columns) { 14 | this.columns = columns; 15 | } 16 | 17 | public int getBuckets() { 18 | return buckets; 19 | } 20 | 21 | public void setBuckets(int buckets) { 22 | this.buckets = buckets; 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/PartitionDesign.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import java.util.List; 4 | 5 | public class PartitionDesign { 6 | private List hash; 7 | private RangePartitionDefinition range; 8 | 9 | public List getHash() { 10 | return hash; 11 | } 12 | 13 | public void setHash(List hash) { 14 | this.hash = hash; 15 | } 16 | 17 | public RangePartitionDefinition getRange() { 18 | return range; 19 | } 20 | 21 | public void setRange(RangePartitionDefinition range) { 22 | this.range = range; 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/RangeBoundValue.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import com.fasterxml.jackson.databind.annotation.JsonDeserialize; 4 | import com.fasterxml.jackson.databind.annotation.JsonSerialize; 5 | import org.weakref.jmx.internal.guava.collect.ImmutableList; 6 | 7 | import java.util.List; 8 | 9 | @JsonDeserialize(using = RangeBoundValueDeserializer.class) 10 | @JsonSerialize(using = RangeBoundValueSerializer.class) 11 | public class RangeBoundValue { 12 | private List values; 13 | 14 | public List getValues() { 15 | return values; 16 | } 17 | 18 | public void setValues(List values) { 19 | this.values = ImmutableList.copyOf(values); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /src/main/java/org/apache/kudu/client/KeyEncoderAccessor.java: -------------------------------------------------------------------------------- 1 | package org.apache.kudu.client; 2 | 3 | import org.apache.kudu.Schema; 4 | 5 | /** 6 | * Little wrapper to access KeyEncoder in Kudu Java client. 7 | */ 8 | public class KeyEncoderAccessor { 9 | private KeyEncoderAccessor() { 10 | } 11 | 12 | public static byte[] encodePrimaryKey(PartialRow row){ 13 | return KeyEncoder.encodePrimaryKey(row); 14 | } 15 | 16 | public static PartialRow decodePrimaryKey(Schema schema, byte[] key) { 17 | return KeyEncoder.decodePrimaryKey(schema, key); 18 | } 19 | 20 | public static byte[] encodeRangePartitionKey(PartialRow row, PartitionSchema.RangeSchema rangeSchema) { 21 | return KeyEncoder.encodeRangePartitionKey(row, rangeSchema); 22 | } 23 | 24 | public static PartialRow decodeRangePartitionKey(Schema schema, PartitionSchema partitionSchema, byte[] key) { 25 | return KeyEncoder.decodeRangePartitionKey(schema, partitionSchema, key); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduTransactionHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 21 | 22 | public enum KuduTransactionHandle implements ConnectorTransactionHandle { 23 | INSTANCE 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/ColumnDesign.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | public class ColumnDesign { 4 | public static final ColumnDesign DEFAULT; 5 | 6 | static { 7 | ColumnDesign design = new ColumnDesign(); 8 | design.setNullable(true); 9 | DEFAULT = design; 10 | } 11 | 12 | private boolean key; 13 | private boolean nullable; 14 | private String encoding; 15 | private String compression; 16 | 17 | public boolean isKey() { 18 | return key; 19 | } 20 | 21 | public void setKey(boolean key) { 22 | this.key = key; 23 | } 24 | 25 | public String getEncoding() { 26 | return encoding; 27 | } 28 | 29 | public void setEncoding(String encoding) { 30 | this.encoding = encoding; 31 | } 32 | 33 | public String getCompression() { 34 | return compression; 35 | } 36 | 37 | public void setCompression(String compression) { 38 | this.compression = compression; 39 | } 40 | 41 | public boolean isNullable() { 42 | return nullable; 43 | } 44 | 45 | public void setNullable(boolean nullable) { 46 | this.nullable = nullable; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/properties/RangePartitionSerializationTest.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import com.fasterxml.jackson.databind.ObjectMapper; 4 | import org.testng.Assert; 5 | import org.testng.annotations.Test; 6 | 7 | import java.io.IOException; 8 | 9 | public class RangePartitionSerializationTest { 10 | private String[] testInputs = new String[]{ 11 | "{\"lower\":1,\"upper\":null}", 12 | "{\"lower\":12345678901234567890,\"upper\":1.234567890123457E-13}", 13 | "{\"lower\":\"abc\",\"upper\":\"abf\"}", 14 | "{\"lower\":false,\"upper\":true}", 15 | "{\"lower\":\"ABCD\",\"upper\":\"ABCDEF\"}", 16 | "{\"lower\":[\"ABCD\",1,0],\"upper\":[\"ABCD\",13,0]}", 17 | }; 18 | 19 | @Test 20 | public void testDeserializationSerialization() throws IOException { 21 | ObjectMapper mapper = new ObjectMapper(); 22 | 23 | for (String input : testInputs) { 24 | RangePartition partition = mapper.readValue(input, RangePartition.class); 25 | 26 | String serialized = mapper.writeValueAsString(partition); 27 | Assert.assertEquals(serialized, input); 28 | } 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduPlugin.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.Plugin; 21 | import com.facebook.presto.spi.connector.ConnectorFactory; 22 | import com.google.common.collect.ImmutableList; 23 | 24 | public class KuduPlugin implements Plugin { 25 | 26 | @Override public Iterable getConnectorFactories() { 27 | return ImmutableList.of(new KuduConnectorFactory("kudu")); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestKuduPlugin.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | 17 | import com.facebook.presto.spi.Plugin; 18 | import com.facebook.presto.spi.connector.ConnectorFactory; 19 | import com.facebook.presto.testing.TestingConnectorContext; 20 | import com.google.common.collect.ImmutableMap; 21 | import org.testng.annotations.Test; 22 | 23 | import static com.google.common.collect.Iterables.getOnlyElement; 24 | 25 | public class TestKuduPlugin { 26 | @Test 27 | public void testCreateConnector() 28 | throws Exception { 29 | Plugin plugin = new KuduPlugin(); 30 | ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); 31 | factory.create("test", ImmutableMap.of("kudu.client.master-addresses", "localhost:8050"), new TestingConnectorContext()); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/SchemaAlreadyExistsException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.spi.PrestoException; 17 | 18 | import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; 19 | import static java.lang.String.format; 20 | 21 | public class SchemaAlreadyExistsException 22 | extends PrestoException 23 | { 24 | private final String schemaName; 25 | 26 | public SchemaAlreadyExistsException(String schemaName) 27 | { 28 | this(schemaName, format("Schema already exists: '%s'", schemaName)); 29 | } 30 | 31 | public SchemaAlreadyExistsException(String schemaName, String message) 32 | { 33 | super(ALREADY_EXISTS, message); 34 | this.schemaName = schemaName; 35 | } 36 | 37 | public String getSchemaName() 38 | { 39 | return schemaName; 40 | } 41 | } -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/Types.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import java.util.Locale; 21 | 22 | import static com.google.common.base.Preconditions.checkArgument; 23 | import static java.util.Objects.requireNonNull; 24 | 25 | public class Types { 26 | private Types() { 27 | } 28 | 29 | public static B checkType(A value, Class target, String name) { 30 | requireNonNull(value, String.format(Locale.ENGLISH, "%s is null", name)); 31 | checkArgument(target.isInstance(value), "%s must be of type %s, not %s", name, target.getName(), 32 | value.getClass().getName()); 33 | return target.cast(value); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduConnectorId.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.google.inject.Inject; 21 | 22 | import java.util.Objects; 23 | 24 | import static java.util.Objects.requireNonNull; 25 | 26 | public class KuduConnectorId { 27 | private final String id; 28 | 29 | @Inject public KuduConnectorId(String id) { 30 | this.id = requireNonNull(id, "id is null"); 31 | } 32 | 33 | @Override public String toString() { 34 | return id; 35 | } 36 | 37 | @Override public int hashCode() { 38 | return Objects.hash(id); 39 | } 40 | 41 | @Override public boolean equals(Object obj) { 42 | if (this == obj) { 43 | return true; 44 | } 45 | 46 | if ((obj == null) || (getClass() != obj.getClass())) { 47 | return false; 48 | } 49 | 50 | return Objects.equals(this.id, ((KuduConnectorId) obj).id); 51 | } 52 | } -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduRecordCursorWithVirtualRowId.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.type.Type; 4 | import io.airlift.slice.Slice; 5 | import io.airlift.slice.Slices; 6 | import org.apache.kudu.Schema; 7 | import org.apache.kudu.client.KeyEncoderAccessor; 8 | import org.apache.kudu.client.KuduScanner; 9 | import org.apache.kudu.client.KuduTable; 10 | import org.apache.kudu.client.PartialRow; 11 | 12 | import java.util.List; 13 | import java.util.Map; 14 | 15 | public class KuduRecordCursorWithVirtualRowId extends KuduRecordCursor { 16 | private final KuduTable table; 17 | private final Map fieldMapping; 18 | 19 | public KuduRecordCursorWithVirtualRowId(KuduScanner scanner, KuduTable table, 20 | List columnTypes, 21 | Map fieldMapping) { 22 | super(scanner, columnTypes); 23 | this.table = table; 24 | this.fieldMapping = fieldMapping; 25 | } 26 | 27 | @Override 28 | protected int mapping(int field) { 29 | return fieldMapping.get(field); 30 | } 31 | 32 | @Override 33 | public Slice getSlice(int field) { 34 | if (fieldMapping.get(field) == -1) { 35 | PartialRow partialRow = buildPrimaryKey(); 36 | return Slices.wrappedBuffer(KeyEncoderAccessor.encodePrimaryKey(partialRow)); 37 | } else { 38 | return super.getSlice(field); 39 | } 40 | } 41 | 42 | private PartialRow buildPrimaryKey() { 43 | Schema schema = table.getSchema(); 44 | PartialRow row = new PartialRow(schema); 45 | RowHelper.copyPrimaryKey(schema, currentRow, row); 46 | return row; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduInsertTableHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.spi.ConnectorInsertTableHandle; 17 | import com.facebook.presto.spi.SchemaTableName; 18 | import com.facebook.presto.spi.type.Type; 19 | import com.fasterxml.jackson.annotation.JsonCreator; 20 | import com.fasterxml.jackson.annotation.JsonProperty; 21 | import org.apache.kudu.client.KuduTable; 22 | 23 | import java.util.List; 24 | 25 | public class KuduInsertTableHandle extends KuduExtendedTableHandle 26 | implements ConnectorInsertTableHandle { 27 | @JsonCreator 28 | public KuduInsertTableHandle( 29 | @JsonProperty("connectorId") String connectorId, 30 | @JsonProperty("schemaTableName") SchemaTableName schemaTableName, 31 | @JsonProperty("columnNames") List columnNames, 32 | @JsonProperty("columnTypes") List columnTypes) { 33 | super(connectorId, schemaTableName, columnNames, columnTypes, null); 34 | } 35 | 36 | public KuduInsertTableHandle(String connectorId, SchemaTableName schemaTableName, 37 | List columnNames, List columnTypes, KuduTable table) { 38 | super(connectorId, schemaTableName, columnNames, columnTypes, null); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestSchemaNoExist.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.testing.QueryRunner; 4 | import com.facebook.presto.tests.AbstractTestQueryFramework; 5 | import org.testng.Assert; 6 | import org.testng.annotations.AfterClass; 7 | import org.testng.annotations.BeforeClass; 8 | import org.testng.annotations.Test; 9 | public class TestSchemaNoExist extends AbstractTestQueryFramework { 10 | private QueryRunner queryRunner; 11 | 12 | private static final String SCHEMA_NAME = "test_presto_schema"; 13 | 14 | private static final String DROP_SCHEMA = "drop schema if exists kudu." + SCHEMA_NAME; 15 | 16 | private static final String CREATE_TABLE = "create table if not exists kudu." + SCHEMA_NAME + ".test_presto_table " + 17 | "(user_id int, user_name varchar) " + 18 | "with(column_design = '{\"user_id\": {\"key\": true}}'," + 19 | "partition_design = '{\"hash\":[{\"columns\":[\"user_id\"], \"buckets\": 2}]}'," + 20 | "num_replicas = 1)"; 21 | 22 | private static final String DROP_TABLE = "drop table if exists kudu." + SCHEMA_NAME + ".test_presto_table"; 23 | 24 | public TestSchemaNoExist() { 25 | super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("test_dummy")); 26 | } 27 | 28 | @Test 29 | public void testCreateTableWithoutSchema() { 30 | try { 31 | queryRunner.execute(CREATE_TABLE); 32 | Assert.fail(); 33 | } catch (Exception e) { 34 | Assert.assertEquals("Schema " + SCHEMA_NAME + " not found", e.getMessage()); 35 | } 36 | } 37 | 38 | @BeforeClass 39 | public void setUp() { 40 | queryRunner = getQueryRunner(); 41 | } 42 | 43 | @AfterClass(alwaysRun = true) 44 | public final void destroy() { 45 | queryRunner.execute(DROP_TABLE); 46 | queryRunner.execute(DROP_SCHEMA); 47 | queryRunner.close(); 48 | queryRunner = null; 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduClientSession.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.ColumnMetadata; 4 | import com.facebook.presto.spi.ConnectorTableMetadata; 5 | import com.facebook.presto.spi.SchemaTableName; 6 | import ml.littlebulb.presto.kudu.properties.RangePartition; 7 | import org.apache.kudu.Schema; 8 | import org.apache.kudu.client.KuduScanner; 9 | import org.apache.kudu.client.KuduSession; 10 | import org.apache.kudu.client.KuduTable; 11 | 12 | import java.util.List; 13 | import java.util.Map; 14 | 15 | public interface KuduClientSession { 16 | List listSchemaNames(); 17 | 18 | List listTables(String schemaNameOrNull); 19 | 20 | Schema getTableSchema(KuduTableHandle tableName); 21 | 22 | boolean tableExists(SchemaTableName schemaTableName); 23 | 24 | Map getTableProperties(KuduTableHandle tableName); 25 | 26 | List buildKuduSplits(KuduTableLayoutHandle layoutHandle); 27 | 28 | KuduScanner createScanner(KuduSplit kuduSplit); 29 | 30 | KuduTable openTable(SchemaTableName schemaTableName); 31 | 32 | KuduSession newSession(); 33 | 34 | void createSchema(String schemaName); 35 | 36 | boolean schemaExists(String schemaName); 37 | 38 | void dropSchema(String schemaName); 39 | 40 | void dropTable(SchemaTableName schemaTableName); 41 | 42 | void renameTable(SchemaTableName schemaTableName, SchemaTableName newSchemaTableName); 43 | 44 | KuduTable createTable(ConnectorTableMetadata tableMetadata, boolean ignoreExisting); 45 | 46 | void addColumn(SchemaTableName schemaTableName, ColumnMetadata column); 47 | 48 | void dropColumn(SchemaTableName schemaTableName, String name); 49 | 50 | void renameColumn(SchemaTableName schemaTableName, String oldName, String newName); 51 | 52 | void addRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition); 53 | 54 | void dropRangePartition(SchemaTableName schemaTableName, RangePartition rangePartition); 55 | } 56 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/RangeBoundValueDeserializer.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import com.fasterxml.jackson.core.JsonParser; 4 | import com.fasterxml.jackson.databind.DeserializationContext; 5 | import com.fasterxml.jackson.databind.JsonDeserializer; 6 | import com.fasterxml.jackson.databind.JsonNode; 7 | import com.google.common.collect.ImmutableList; 8 | 9 | import java.io.IOException; 10 | import java.util.ArrayList; 11 | import java.util.Iterator; 12 | 13 | public class RangeBoundValueDeserializer extends JsonDeserializer { 14 | @Override 15 | public RangeBoundValue deserialize(JsonParser jp, DeserializationContext ctxt) 16 | throws IOException { 17 | JsonNode node = jp.getCodec().readTree(jp); 18 | 19 | if (node.isNull()) { 20 | return null; 21 | } else { 22 | RangeBoundValue value = new RangeBoundValue(); 23 | if (node.isArray()) { 24 | ArrayList list = new ArrayList<>(); 25 | Iterator iter = node.elements(); 26 | while (iter.hasNext()) { 27 | Object v = toValue(iter.next()); 28 | list.add(v); 29 | } 30 | value.setValues(ImmutableList.copyOf(list)); 31 | } else { 32 | Object v = toValue(node); 33 | value.setValues(ImmutableList.of(v)); 34 | } 35 | return value; 36 | } 37 | } 38 | 39 | private Object toValue(JsonNode node) throws IOException { 40 | if (node.isTextual()) { 41 | return node.asText(); 42 | } else if (node.isNumber()) { 43 | return node.numberValue(); 44 | } else if (node.isBoolean()) { 45 | return node.asBoolean(); 46 | } else if (node.isBinary()) { 47 | return node.binaryValue(); 48 | } else { 49 | throw new IllegalStateException("Unexpected range bound value: " + node); 50 | } 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduPageSinkProvider.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.ConnectorInsertTableHandle; 4 | import com.facebook.presto.spi.ConnectorOutputTableHandle; 5 | import com.facebook.presto.spi.ConnectorPageSink; 6 | import com.facebook.presto.spi.ConnectorSession; 7 | import com.facebook.presto.spi.connector.ConnectorPageSinkProvider; 8 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 9 | 10 | import javax.inject.Inject; 11 | 12 | import static com.google.common.base.Preconditions.checkArgument; 13 | import static java.util.Objects.requireNonNull; 14 | 15 | public class KuduPageSinkProvider implements ConnectorPageSinkProvider { 16 | private final KuduClientSession clientSession; 17 | 18 | @Inject 19 | public KuduPageSinkProvider(KuduClientSession clientSession) { 20 | this.clientSession = requireNonNull(clientSession, "clientSession is null"); 21 | } 22 | 23 | @Override 24 | public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle) { 25 | requireNonNull(outputTableHandle, "outputTableHandle is null"); 26 | checkArgument(outputTableHandle instanceof KuduOutputTableHandle, "outputTableHandle is not an instance of KuduOutputTableHandle"); 27 | KuduOutputTableHandle handle = (KuduOutputTableHandle) outputTableHandle; 28 | 29 | return new KuduPageSink(session, clientSession, handle, handle.isGenerateUUID()); 30 | } 31 | 32 | @Override 33 | public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle) { 34 | requireNonNull(insertTableHandle, "insertTableHandle is null"); 35 | checkArgument(insertTableHandle instanceof KuduInsertTableHandle, "insertTableHandle is not an instance of KuduInsertTableHandle"); 36 | KuduInsertTableHandle handle = (KuduInsertTableHandle) insertTableHandle; 37 | 38 | return new KuduPageSink(session, clientSession, handle, false); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduExtendedTableHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.spi.SchemaTableName; 17 | import com.facebook.presto.spi.type.Type; 18 | import com.fasterxml.jackson.annotation.JsonProperty; 19 | import com.google.common.collect.ImmutableList; 20 | import org.apache.kudu.client.KuduTable; 21 | 22 | import java.util.List; 23 | 24 | import static com.google.common.base.Preconditions.checkArgument; 25 | import static java.util.Objects.requireNonNull; 26 | 27 | public abstract class KuduExtendedTableHandle extends KuduTableHandle { 28 | private final List columnNames; 29 | private final List columnTypes; 30 | 31 | public KuduExtendedTableHandle(String connectorId, SchemaTableName schemaTableName, 32 | List columnNames, List columnTypes, 33 | KuduTable table) { 34 | super(connectorId, schemaTableName, table); 35 | 36 | requireNonNull(columnNames, "columnNames is null"); 37 | requireNonNull(columnTypes, "columnTypes is null"); 38 | checkArgument(columnNames.size() == columnTypes.size(), "columnNames and columnTypes sizes don't match"); 39 | this.columnNames = ImmutableList.copyOf(columnNames); 40 | this.columnTypes = ImmutableList.copyOf(columnTypes); 41 | } 42 | 43 | @JsonProperty 44 | public List getColumnNames() { 45 | return columnNames; 46 | } 47 | 48 | @JsonProperty 49 | public List getColumnTypes() { 50 | return columnTypes; 51 | } 52 | 53 | public List getOriginalColumnTypes() { 54 | return columnTypes; 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduPageSourceProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ConnectorPageSource; 22 | import com.facebook.presto.spi.ConnectorSession; 23 | import com.facebook.presto.spi.ConnectorSplit; 24 | import com.facebook.presto.spi.RecordPageSource; 25 | import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; 26 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 27 | import com.google.inject.Inject; 28 | 29 | import java.util.List; 30 | 31 | import static java.util.Objects.requireNonNull; 32 | 33 | public class KuduPageSourceProvider implements ConnectorPageSourceProvider { 34 | 35 | private KuduRecordSetProvider recordSetProvider; 36 | 37 | @Inject 38 | public KuduPageSourceProvider(KuduRecordSetProvider recordSetProvider) { 39 | this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); 40 | } 41 | 42 | @Override 43 | public ConnectorPageSource createPageSource(ConnectorTransactionHandle transactionHandle, 44 | ConnectorSession session, ConnectorSplit split, List columns) { 45 | KuduRecordSet recordSet = (KuduRecordSet) recordSetProvider.getRecordSet(transactionHandle, session, split, columns); 46 | if (columns.contains(KuduColumnHandle.ROW_ID_HANDLE)) { 47 | return new KuduUpdatablePageSource(recordSet); 48 | } else { 49 | return new RecordPageSource(recordSet); 50 | } 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduSplitManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ConnectorSession; 21 | import com.facebook.presto.spi.ConnectorSplitSource; 22 | import com.facebook.presto.spi.ConnectorTableLayoutHandle; 23 | import com.facebook.presto.spi.FixedSplitSource; 24 | import com.facebook.presto.spi.connector.ConnectorSplitManager; 25 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 26 | 27 | import java.util.List; 28 | 29 | import javax.inject.Inject; 30 | 31 | import static java.util.Objects.requireNonNull; 32 | 33 | public class KuduSplitManager implements ConnectorSplitManager { 34 | 35 | private final String connectorId; 36 | private final KuduClientSession clientSession; 37 | 38 | @Inject 39 | public KuduSplitManager(KuduConnectorId connectorId, KuduClientSession clientSession) { 40 | this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); 41 | this.clientSession = requireNonNull(clientSession, "clientSession is null"); 42 | } 43 | 44 | @Override 45 | public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, 46 | ConnectorSession session, ConnectorTableLayoutHandle layout, 47 | SplitSchedulingStrategy splitSchedulingStrategy) { 48 | KuduTableLayoutHandle layoutHandle = (KuduTableLayoutHandle) layout; 49 | 50 | List splits = clientSession.buildKuduSplits(layoutHandle); 51 | 52 | return new FixedSplitSource(splits); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduRecordSetProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ConnectorSession; 22 | import com.facebook.presto.spi.ConnectorSplit; 23 | import com.facebook.presto.spi.RecordSet; 24 | import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; 25 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 26 | 27 | import java.util.List; 28 | 29 | import javax.inject.Inject; 30 | 31 | import static java.util.Objects.requireNonNull; 32 | import static ml.littlebulb.presto.kudu.Types.checkType; 33 | 34 | public class KuduRecordSetProvider implements ConnectorRecordSetProvider { 35 | 36 | private final String connectorId; 37 | private final KuduClientSession clientSession; 38 | 39 | @Inject 40 | public KuduRecordSetProvider(KuduConnectorId connectorId, KuduClientSession clientSession) { 41 | this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); 42 | this.clientSession = clientSession; 43 | } 44 | 45 | @Override 46 | public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, 47 | ConnectorSession session, ConnectorSplit split, List columns) { 48 | requireNonNull(split, "split is null"); 49 | requireNonNull(columns, "columns is null"); 50 | 51 | KuduSplit kuduSplit = checkType(split, KuduSplit.class, "split is not class KuduSplit"); 52 | 53 | return new KuduRecordSet(clientSession, kuduSplit, columns); 54 | } 55 | 56 | public KuduClientSession getClientSession() { 57 | return clientSession; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/RangeBoundValueSerializer.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.properties; 2 | 3 | import com.fasterxml.jackson.core.JsonGenerator; 4 | import com.fasterxml.jackson.databind.JsonSerializer; 5 | import com.fasterxml.jackson.databind.SerializerProvider; 6 | 7 | import java.io.IOException; 8 | import java.math.BigDecimal; 9 | import java.math.BigInteger; 10 | 11 | public class RangeBoundValueSerializer extends JsonSerializer { 12 | @Override 13 | public void serialize(Object value, JsonGenerator gen, SerializerProvider serializers) 14 | throws IOException { 15 | if (value == null) { 16 | gen.writeNull(); 17 | } else { 18 | RangeBoundValue rbv = (RangeBoundValue) value; 19 | if (rbv.getValues().size() == 1) { 20 | writeValue(rbv.getValues().get(0), gen); 21 | } else { 22 | gen.writeStartArray(rbv.getValues().size()); 23 | for (Object obj: rbv.getValues()) { 24 | writeValue(obj, gen); 25 | } 26 | gen.writeEndArray(); 27 | } 28 | } 29 | } 30 | 31 | private void writeValue(Object obj, JsonGenerator gen) throws IOException { 32 | if (obj == null) { 33 | throw new IllegalStateException("Unexpected null value"); 34 | } else if (obj instanceof String) { 35 | gen.writeString((String) obj); 36 | } else if (Number.class.isAssignableFrom(obj.getClass())) { 37 | if (obj instanceof Long) { 38 | gen.writeNumber((Long) obj); 39 | } else if (obj instanceof Integer) { 40 | gen.writeNumber((Integer) obj); 41 | } else if (obj instanceof Short) { 42 | gen.writeNumber((Short) obj); 43 | } else if (obj instanceof Double) { 44 | gen.writeNumber((Double) obj); 45 | } else if (obj instanceof Float) { 46 | gen.writeNumber((Float) obj); 47 | } else if (obj instanceof BigInteger) { 48 | gen.writeNumber((BigInteger) obj); 49 | } else if (obj instanceof BigDecimal) { 50 | gen.writeNumber((BigDecimal) obj); 51 | } else { 52 | throw new IllegalStateException("Unknown number value: " + obj); 53 | } 54 | } else if (obj instanceof Boolean) { 55 | gen.writeBoolean((Boolean) obj); 56 | } else if (obj instanceof byte[]) { 57 | gen.writeBinary((byte[]) obj); 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduHandleResolver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ConnectorHandleResolver; 22 | import com.facebook.presto.spi.ConnectorInsertTableHandle; 23 | import com.facebook.presto.spi.ConnectorOutputTableHandle; 24 | import com.facebook.presto.spi.ConnectorSplit; 25 | import com.facebook.presto.spi.ConnectorTableHandle; 26 | import com.facebook.presto.spi.ConnectorTableLayoutHandle; 27 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 28 | 29 | public class KuduHandleResolver implements ConnectorHandleResolver { 30 | @Override 31 | public Class getTableHandleClass() { 32 | return KuduTableHandle.class; 33 | } 34 | 35 | @Override 36 | public Class getTableLayoutHandleClass() { 37 | return KuduTableLayoutHandle.class; 38 | } 39 | 40 | @Override 41 | public Class getColumnHandleClass() { 42 | return KuduColumnHandle.class; 43 | } 44 | 45 | @Override 46 | public Class getSplitClass() { 47 | return KuduSplit.class; 48 | } 49 | 50 | @Override 51 | public Class getTransactionHandleClass() { 52 | return KuduTransactionHandle.class; 53 | } 54 | 55 | @Override 56 | public Class getInsertTableHandleClass() { 57 | return KuduInsertTableHandle.class; 58 | } 59 | 60 | @Override 61 | public Class getOutputTableHandleClass() { 62 | return KuduOutputTableHandle.class; 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduConnectorFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ConnectorHandleResolver; 21 | import com.facebook.presto.spi.connector.Connector; 22 | import com.facebook.presto.spi.connector.ConnectorContext; 23 | import com.facebook.presto.spi.connector.ConnectorFactory; 24 | import com.google.inject.Injector; 25 | import io.airlift.bootstrap.Bootstrap; 26 | import io.airlift.json.JsonModule; 27 | 28 | import java.util.Map; 29 | 30 | import static java.util.Objects.requireNonNull; 31 | 32 | public class KuduConnectorFactory implements ConnectorFactory { 33 | 34 | private final String name; 35 | 36 | public KuduConnectorFactory(String connectorName) { 37 | this.name = connectorName; 38 | } 39 | 40 | @Override 41 | public String getName() { 42 | return name; 43 | } 44 | 45 | @Override 46 | public ConnectorHandleResolver getHandleResolver() { 47 | return new KuduHandleResolver(); 48 | } 49 | 50 | @Override 51 | public Connector create(String connectorId, Map config, 52 | ConnectorContext context) { 53 | requireNonNull(config, "config is null"); 54 | 55 | try { 56 | Bootstrap app = new Bootstrap(new JsonModule(), 57 | new KuduModule(connectorId, context.getTypeManager())); 58 | 59 | Injector injector = 60 | app.strictConfig().doNotInitializeLogging().setRequiredConfigurationProperties(config) 61 | .initialize(); 62 | 63 | return injector.getInstance(KuduConnector.class); 64 | } catch (Exception e) { 65 | throw new RuntimeException(e); 66 | } 67 | } 68 | } -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduSplit.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ConnectorSplit; 21 | import com.facebook.presto.spi.HostAddress; 22 | import com.fasterxml.jackson.annotation.JsonCreator; 23 | import com.fasterxml.jackson.annotation.JsonProperty; 24 | import com.google.common.collect.ImmutableList; 25 | 26 | import java.util.List; 27 | 28 | import static java.util.Objects.requireNonNull; 29 | 30 | public class KuduSplit implements ConnectorSplit { 31 | 32 | private final KuduTableHandle tableHandle; 33 | private final int primaryKeyColumnCount; 34 | private final byte[] pb; 35 | 36 | @JsonCreator 37 | public KuduSplit(@JsonProperty("tableHandle") KuduTableHandle tableHandle, 38 | @JsonProperty("primaryKeyColumnCount") int primaryKeyColumnCount, 39 | @JsonProperty("pb") byte[] pb) { 40 | this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); 41 | this.primaryKeyColumnCount = primaryKeyColumnCount; 42 | this.pb = requireNonNull(pb, "pb is null"); 43 | } 44 | 45 | @JsonProperty 46 | public KuduTableHandle getTableHandle() { return tableHandle; } 47 | 48 | @JsonProperty 49 | public byte[] getPb() { 50 | return pb; 51 | } 52 | 53 | @JsonProperty 54 | public int getPrimaryKeyColumnCount() { 55 | return primaryKeyColumnCount; 56 | } 57 | 58 | @Override 59 | public boolean isRemotelyAccessible() { 60 | return true; 61 | } 62 | 63 | @Override 64 | public List getAddresses() { 65 | return ImmutableList.of(); 66 | } 67 | 68 | @Override 69 | public Object getInfo() { 70 | return this; 71 | } 72 | } 73 | 74 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduOutputTableHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.spi.ConnectorOutputTableHandle; 17 | import com.facebook.presto.spi.SchemaTableName; 18 | import com.facebook.presto.spi.type.Type; 19 | import com.fasterxml.jackson.annotation.JsonCreator; 20 | import com.fasterxml.jackson.annotation.JsonProperty; 21 | import com.google.common.collect.ImmutableList; 22 | import org.apache.kudu.client.KuduTable; 23 | 24 | import java.util.List; 25 | 26 | public class KuduOutputTableHandle extends KuduExtendedTableHandle 27 | implements ConnectorOutputTableHandle { 28 | private final boolean generateUUID; 29 | private final List originalColumnTypes; 30 | 31 | @JsonCreator 32 | public KuduOutputTableHandle( 33 | @JsonProperty("connectorId") String connectorId, 34 | @JsonProperty("schemaTableName") SchemaTableName schemaTableName, 35 | @JsonProperty("originalColumnTypes") List originalColumnTypes, 36 | @JsonProperty("columnNames") List columnNames, 37 | @JsonProperty("columnTypes") List columnTypes, 38 | @JsonProperty("generateUUID") boolean generateUUID) { 39 | this(connectorId, schemaTableName, originalColumnTypes, columnNames, columnTypes, generateUUID, null); 40 | } 41 | 42 | public KuduOutputTableHandle(String connectorId, SchemaTableName schemaTableName, 43 | List originalColumnTypes, 44 | List columnNames, List columnTypes, 45 | boolean generateUUID, KuduTable table) { 46 | super(connectorId, schemaTableName, columnNames, columnTypes, table); 47 | this.originalColumnTypes = ImmutableList.copyOf(originalColumnTypes); 48 | this.generateUUID = generateUUID; 49 | } 50 | 51 | @JsonProperty 52 | public boolean isGenerateUUID() { 53 | return generateUUID; 54 | } 55 | 56 | @JsonProperty 57 | @Override 58 | public List getOriginalColumnTypes() { 59 | return originalColumnTypes; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/procedures/RangePartitionProcedures.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu.procedures; 2 | 3 | import com.facebook.presto.spi.SchemaTableName; 4 | import com.facebook.presto.spi.procedure.Procedure; 5 | import com.facebook.presto.spi.procedure.Procedure.Argument; 6 | import com.google.common.collect.ImmutableList; 7 | import ml.littlebulb.presto.kudu.KuduClientSession; 8 | import ml.littlebulb.presto.kudu.properties.KuduTableProperties; 9 | import ml.littlebulb.presto.kudu.properties.RangePartition; 10 | 11 | import java.lang.invoke.MethodHandle; 12 | 13 | import javax.inject.Inject; 14 | 15 | import static com.facebook.presto.spi.block.MethodHandleUtil.methodHandle; 16 | import static com.facebook.presto.spi.type.StandardTypes.VARCHAR; 17 | import static java.util.Objects.requireNonNull; 18 | 19 | public class RangePartitionProcedures { 20 | private static final MethodHandle ADD = methodHandle(RangePartitionProcedures.class, "addRangePartition", 21 | String.class, String.class, String.class); 22 | private static final MethodHandle DROP = methodHandle(RangePartitionProcedures.class, "dropRangePartition", 23 | String.class, String.class, String.class); 24 | 25 | private final KuduClientSession clientSession; 26 | 27 | @Inject 28 | public RangePartitionProcedures(KuduClientSession clientSession) { 29 | this.clientSession = requireNonNull(clientSession); 30 | } 31 | 32 | public Procedure getAddPartitionProcedure() { 33 | return new Procedure( 34 | "system", 35 | "add_range_partition", 36 | ImmutableList.of(new Argument("schema", VARCHAR), new Argument("table", VARCHAR), 37 | new Argument("range_bounds", VARCHAR)), 38 | ADD.bindTo(this)); 39 | } 40 | 41 | public Procedure getDropPartitionProcedure() { 42 | return new Procedure( 43 | "system", 44 | "drop_range_partition", 45 | ImmutableList.of(new Argument("schema", VARCHAR), new Argument("table", VARCHAR), 46 | new Argument("range_bounds", VARCHAR)), 47 | DROP.bindTo(this)); 48 | } 49 | 50 | public void addRangePartition(String schema, String table, String rangeBounds) { 51 | SchemaTableName schemaTableName = new SchemaTableName(schema, table); 52 | RangePartition rangePartition = KuduTableProperties.parseRangePartition(rangeBounds); 53 | clientSession.addRangePartition(schemaTableName, rangePartition); 54 | } 55 | 56 | public void dropRangePartition(String schema, String table, String rangeBounds) { 57 | SchemaTableName schemaTableName = new SchemaTableName(schema, table); 58 | RangePartition rangePartition = KuduTableProperties.parseRangePartition(rangeBounds); 59 | clientSession.dropRangePartition(schemaTableName, rangePartition); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestKuduIntegrationSmokeTest.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.type.VarcharType; 4 | import com.facebook.presto.testing.MaterializedResult; 5 | import com.facebook.presto.testing.MaterializedRow; 6 | import com.facebook.presto.testing.QueryRunner; 7 | import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; 8 | import org.testng.Assert; 9 | import org.testng.annotations.AfterClass; 10 | import org.testng.annotations.BeforeClass; 11 | import org.testng.annotations.Test; 12 | 13 | import static io.airlift.tpch.TpchTable.ORDERS; 14 | 15 | /** 16 | * Kudu master server is expected to be running on localhost. At least one 17 | * Kudu tablet server must be running, too. 18 | * With Docker, use e.g. 19 | * "docker run --rm -d --name apache-kudu --net=host usuresearch/kudu-docker-slim:release-v1.6.0-2" 20 | */ 21 | public class TestKuduIntegrationSmokeTest extends AbstractTestIntegrationSmokeTest { 22 | public static final String SCHEMA = "tpch"; 23 | 24 | private QueryRunner queryRunner; 25 | 26 | public TestKuduIntegrationSmokeTest() { 27 | super(() -> KuduQueryRunnerFactory.createKuduQueryRunnerTpch(ORDERS)); 28 | } 29 | 30 | @BeforeClass 31 | public void setUp() { 32 | queryRunner = getQueryRunner(); 33 | } 34 | 35 | /** 36 | * Overrides original implementation because of usage of 'extra' column. 37 | */ 38 | @Test 39 | @Override 40 | public void testDescribeTable() { 41 | MaterializedResult actualColumns = this.computeActual("DESC ORDERS").toTestTypes(); 42 | MaterializedResult.Builder builder = MaterializedResult.resultBuilder(this.getQueryRunner().getDefaultSession(), VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR); 43 | for (MaterializedRow row: actualColumns.getMaterializedRows()) { 44 | builder.row(row.getField(0), row.getField(1), "", ""); 45 | } 46 | MaterializedResult filteredActual = builder.build(); 47 | builder = MaterializedResult.resultBuilder(this.getQueryRunner().getDefaultSession(), VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR, VarcharType.VARCHAR); 48 | MaterializedResult expectedColumns = builder 49 | .row("orderkey", "bigint", "", "") 50 | .row("custkey", "bigint", "", "") 51 | .row("orderstatus", "varchar", "", "") 52 | .row("totalprice", "double", "", "") 53 | .row("orderdate", "varchar", "", "") 54 | .row("orderpriority", "varchar", "", "") 55 | .row("clerk", "varchar", "", "") 56 | .row("shippriority", "integer", "", "") 57 | .row("comment", "varchar", "", "").build(); 58 | Assert.assertEquals(filteredActual, expectedColumns, String.format("%s != %s", filteredActual, expectedColumns)); 59 | } 60 | 61 | 62 | @AfterClass(alwaysRun = true) 63 | public final void destroy() { 64 | if (queryRunner != null) { 65 | queryRunner.close(); 66 | queryRunner = null; 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduUpdatablePageSource.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.Page; 4 | import com.facebook.presto.spi.RecordPageSource; 5 | import com.facebook.presto.spi.UpdatablePageSource; 6 | import com.facebook.presto.spi.block.Block; 7 | import io.airlift.slice.Slice; 8 | import org.apache.kudu.Schema; 9 | import org.apache.kudu.client.Delete; 10 | import org.apache.kudu.client.KeyEncoderAccessor; 11 | import org.apache.kudu.client.KuduException; 12 | import org.apache.kudu.client.KuduSession; 13 | import org.apache.kudu.client.KuduTable; 14 | import org.apache.kudu.client.PartialRow; 15 | import org.apache.kudu.client.SessionConfiguration.FlushMode; 16 | 17 | import java.io.IOException; 18 | import java.util.Collection; 19 | import java.util.Collections; 20 | import java.util.concurrent.CompletableFuture; 21 | 22 | public class KuduUpdatablePageSource implements UpdatablePageSource { 23 | private final KuduClientSession clientSession; 24 | private final KuduTable table; 25 | private final RecordPageSource inner; 26 | 27 | public KuduUpdatablePageSource(KuduRecordSet recordSet) { 28 | this.clientSession = recordSet.getClientSession(); 29 | this.table = recordSet.getTable(); 30 | this.inner = new RecordPageSource(recordSet); 31 | } 32 | 33 | @Override 34 | public void deleteRows(Block rowIds) { 35 | Schema schema = table.getSchema(); 36 | KuduSession session = clientSession.newSession(); 37 | session.setFlushMode(FlushMode.AUTO_FLUSH_BACKGROUND); 38 | try { 39 | try { 40 | for (int i = 0; i < rowIds.getPositionCount(); i++) { 41 | int len = rowIds.getSliceLength(i); 42 | Slice slice = rowIds.getSlice(i, 0, len); 43 | PartialRow row = KeyEncoderAccessor.decodePrimaryKey(schema, slice.getBytes()); 44 | Delete delete = table.newDelete(); 45 | RowHelper.copyPrimaryKey(schema, row, delete.getRow()); 46 | session.apply(delete); 47 | } 48 | } finally { 49 | session.close(); 50 | } 51 | } catch (KuduException e) { 52 | throw new RuntimeException(e); 53 | } 54 | } 55 | 56 | @Override 57 | public CompletableFuture> finish() { 58 | CompletableFuture> cf = new CompletableFuture<>(); 59 | cf.complete(Collections.emptyList()); 60 | return cf; 61 | } 62 | 63 | @Override 64 | public long getCompletedBytes() { 65 | return inner.getCompletedBytes(); 66 | } 67 | 68 | @Override 69 | public long getReadTimeNanos() { 70 | return inner.getReadTimeNanos(); 71 | } 72 | 73 | @Override 74 | public boolean isFinished() { 75 | return inner.isFinished(); 76 | } 77 | 78 | @Override 79 | public Page getNextPage() { 80 | return inner.getNextPage(); 81 | } 82 | 83 | @Override 84 | public long getSystemMemoryUsage() { 85 | return inner.getSystemMemoryUsage(); 86 | } 87 | 88 | @Override 89 | public void close() throws IOException { 90 | inner.close(); 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduTableHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ConnectorTableHandle; 21 | import com.facebook.presto.spi.SchemaTableName; 22 | import com.fasterxml.jackson.annotation.JsonCreator; 23 | import com.fasterxml.jackson.annotation.JsonProperty; 24 | import org.apache.kudu.client.KuduTable; 25 | 26 | import java.util.Objects; 27 | 28 | import static java.util.Locale.ENGLISH; 29 | import static java.util.Objects.requireNonNull; 30 | 31 | public class KuduTableHandle implements ConnectorTableHandle { 32 | 33 | private final String connectorId; 34 | private final SchemaTableName schemaTableName; 35 | private transient KuduTable table; 36 | 37 | @JsonCreator 38 | public KuduTableHandle(@JsonProperty("connectorId") String connectorId, 39 | @JsonProperty("schemaTableName") SchemaTableName schemaTableName) { 40 | this(connectorId, schemaTableName, null); 41 | } 42 | 43 | public KuduTableHandle(String connectorId, SchemaTableName schemaTableName, 44 | KuduTable table) { 45 | this.connectorId = requireNonNull(connectorId.toLowerCase(ENGLISH), "connectorId is null"); 46 | this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); 47 | this.table = table; 48 | } 49 | 50 | public KuduTable getTable(KuduClientSession session) { 51 | if (table == null) { 52 | table = session.openTable(schemaTableName); 53 | } 54 | return table; 55 | } 56 | 57 | @JsonProperty 58 | public String getConnectorId() { 59 | return connectorId; 60 | } 61 | 62 | @JsonProperty 63 | public SchemaTableName getSchemaTableName() { 64 | return schemaTableName; 65 | } 66 | 67 | @Override 68 | public int hashCode() { 69 | return Objects.hash(connectorId, schemaTableName); 70 | } 71 | 72 | @Override 73 | public boolean equals(Object obj) { 74 | if (this == obj) { 75 | return true; 76 | } 77 | if ((obj == null) || (getClass() != obj.getClass())) { 78 | return false; 79 | } 80 | 81 | KuduTableHandle other = (KuduTableHandle) obj; 82 | return Objects.equals(this.connectorId, other.connectorId) && this.schemaTableName 83 | .equals(other.getSchemaTableName()); 84 | } 85 | 86 | @Override 87 | public String toString() { 88 | return connectorId + ":" + schemaTableName; 89 | } 90 | 91 | } 92 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestDecimalColumnsTest.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.testing.MaterializedResult; 4 | import com.facebook.presto.testing.QueryRunner; 5 | import com.facebook.presto.tests.AbstractTestQueryFramework; 6 | import org.testng.Assert; 7 | import org.testng.annotations.AfterClass; 8 | import org.testng.annotations.BeforeClass; 9 | import org.testng.annotations.Test; 10 | 11 | class TestDec { 12 | final int precision; 13 | final int scale; 14 | 15 | TestDec(int precision, int scale) { 16 | this.precision = precision; 17 | this.scale = scale; 18 | } 19 | } 20 | 21 | public class TestDecimalColumnsTest extends AbstractTestQueryFramework { 22 | private QueryRunner queryRunner; 23 | 24 | static final TestDec[] testDecList = { 25 | new TestDec(10,0), 26 | new TestDec(15,4), 27 | new TestDec(18,6), 28 | new TestDec(18,7), 29 | new TestDec(19,8), 30 | new TestDec(24,14), 31 | new TestDec(38,20), 32 | new TestDec(38, 28), 33 | }; 34 | 35 | 36 | public TestDecimalColumnsTest() throws Exception { 37 | super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("decimal")); 38 | } 39 | 40 | @Test 41 | public void testCreateTableWithDecimalColumn() { 42 | for (TestDec dec : testDecList) { 43 | doTestCreateTableWithDecimalColumn(dec); 44 | } 45 | } 46 | 47 | 48 | public void doTestCreateTableWithDecimalColumn(TestDec dec) { 49 | String dropTable = "DROP TABLE IF EXISTS test_dec"; 50 | String createTable = "CREATE TABLE test_dec (\n"; 51 | createTable += " id INT,\n"; 52 | createTable += " dec DECIMAL(" + dec.precision + "," + dec.scale + ")\n"; 53 | createTable += ") WITH (\n" + 54 | " column_design = '{\"id\": {\"key\": true}}',\n" + 55 | " partition_design = '{\"hash\":[{\"columns\":[\"id\"], \"buckets\": 2}]}',\n" + 56 | " num_replicas = 1\n" + 57 | ")"; 58 | 59 | queryRunner.execute(dropTable); 60 | queryRunner.execute(createTable); 61 | 62 | String fullPrecisionValue = "1234567890.1234567890123456789012345678"; 63 | int maxScale = dec.precision - 10; 64 | int valuePrecision = dec.precision - maxScale + Math.min(maxScale, dec.scale); 65 | String insertValue = fullPrecisionValue.substring(0, valuePrecision + 1); 66 | queryRunner.execute("INSERT INTO test_dec VALUES(1, DECIMAL '" + insertValue + "')"); 67 | 68 | MaterializedResult result = queryRunner.execute("SELECT id, CAST((dec - (DECIMAL '" + insertValue + "')) as DOUBLE) FROM test_dec"); 69 | Assert.assertEquals(result.getRowCount(), 1); 70 | Object obj = result.getMaterializedRows().get(0).getField(1); 71 | Assert.assertTrue(obj instanceof Double); 72 | Double actual = (Double) obj; 73 | Assert.assertEquals(0, actual, 0.3 * Math.pow(0.1, dec.scale), "p=" + dec.precision + ",s=" + dec.scale + " => " + actual + ",insert = " + insertValue); 74 | } 75 | 76 | @BeforeClass 77 | public void setUp() { 78 | queryRunner = getQueryRunner(); 79 | } 80 | 81 | @AfterClass(alwaysRun = true) 82 | public final void destroy() { 83 | if (queryRunner != null) { 84 | queryRunner.close(); 85 | queryRunner = null; 86 | } 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/RowHelper.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.google.common.base.Charsets; 4 | import org.apache.kudu.Schema; 5 | import org.apache.kudu.client.PartialRow; 6 | import org.apache.kudu.client.RowResult; 7 | 8 | public class RowHelper { 9 | private RowHelper() { 10 | } 11 | 12 | public static void copyPrimaryKey(Schema schema, RowResult from, PartialRow to) { 13 | for (int i = 0; i < schema.getPrimaryKeyColumnCount(); i++) { 14 | switch (schema.getColumnByIndex(i).getType()) { 15 | case STRING: 16 | to.addStringUtf8(i, from.getString(i).getBytes(Charsets.UTF_8)); 17 | break; 18 | case INT64: 19 | case UNIXTIME_MICROS: 20 | to.addLong(i, from.getLong(i)); 21 | break; 22 | case INT32: 23 | to.addInt(i, from.getInt(i)); 24 | break; 25 | case INT16: 26 | to.addShort(i, from.getShort(i)); 27 | break; 28 | case INT8: 29 | to.addByte(i, from.getByte(i)); 30 | break; 31 | case DOUBLE: 32 | to.addDouble(i, from.getDouble(i)); 33 | break; 34 | case FLOAT: 35 | to.addFloat(i, from.getFloat(i)); 36 | break; 37 | case BOOL: 38 | to.addBoolean(i, from.getBoolean(i)); 39 | break; 40 | case BINARY: 41 | to.addBinary(i, from.getBinary(i)); 42 | break; 43 | default: 44 | throw new IllegalStateException("Unknown type " + schema.getColumnByIndex(i).getType() 45 | + " for column " + schema.getColumnByIndex(i).getName()); 46 | } 47 | } 48 | } 49 | 50 | public static void copyPrimaryKey(Schema schema, PartialRow from, PartialRow to) { 51 | for (int i = 0; i < schema.getPrimaryKeyColumnCount(); i++) { 52 | switch (schema.getColumnByIndex(i).getType()) { 53 | case STRING: 54 | to.addStringUtf8(i, from.getString(i).getBytes(Charsets.UTF_8)); 55 | break; 56 | case INT64: 57 | case UNIXTIME_MICROS: 58 | to.addLong(i, from.getLong(i)); 59 | break; 60 | case INT32: 61 | to.addInt(i, from.getInt(i)); 62 | break; 63 | case INT16: 64 | to.addShort(i, from.getShort(i)); 65 | break; 66 | case INT8: 67 | to.addByte(i, from.getByte(i)); 68 | break; 69 | case DOUBLE: 70 | to.addDouble(i, from.getDouble(i)); 71 | break; 72 | case FLOAT: 73 | to.addFloat(i, from.getFloat(i)); 74 | break; 75 | case BOOL: 76 | to.addBoolean(i, from.getBoolean(i)); 77 | break; 78 | case BINARY: 79 | to.addBinary(i, from.getBinary(i)); 80 | break; 81 | default: 82 | throw new IllegalStateException("Unknown type " + schema.getColumnByIndex(i).getType() 83 | + " for column " + schema.getColumnByIndex(i).getName()); 84 | } 85 | } 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduTableLayoutHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ConnectorTableLayoutHandle; 22 | import com.facebook.presto.spi.predicate.TupleDomain; 23 | import com.fasterxml.jackson.annotation.JsonCreator; 24 | import com.fasterxml.jackson.annotation.JsonProperty; 25 | 26 | import java.util.Objects; 27 | import java.util.Optional; 28 | import java.util.Set; 29 | 30 | import static com.google.common.base.MoreObjects.toStringHelper; 31 | import static java.util.Objects.requireNonNull; 32 | 33 | public class KuduTableLayoutHandle implements ConnectorTableLayoutHandle { 34 | private final KuduTableHandle tableHandle; 35 | private final TupleDomain constraintSummary; 36 | private final Optional> desiredColumns; 37 | 38 | @JsonCreator 39 | public KuduTableLayoutHandle(@JsonProperty("tableHandle") KuduTableHandle tableHandle, 40 | @JsonProperty("constraintSummary") TupleDomain constraintSummary, 41 | @JsonProperty("desiredColumns") Optional> desiredColumns) { 42 | this.tableHandle = requireNonNull(tableHandle, "table is null"); 43 | this.constraintSummary = constraintSummary; 44 | this.desiredColumns = desiredColumns; 45 | } 46 | 47 | @JsonProperty 48 | public KuduTableHandle getTableHandle() { 49 | return tableHandle; 50 | } 51 | 52 | @JsonProperty 53 | public TupleDomain getConstraintSummary() { 54 | return constraintSummary; 55 | } 56 | 57 | @JsonProperty 58 | public Optional> getDesiredColumns() { 59 | return desiredColumns; 60 | } 61 | 62 | 63 | @Override 64 | public boolean equals(Object obj) { 65 | if (this == obj) { 66 | return true; 67 | } 68 | 69 | if (obj == null || getClass() != obj.getClass()) { 70 | return false; 71 | } 72 | 73 | KuduTableLayoutHandle other = (KuduTableLayoutHandle) obj; 74 | return Objects.equals(tableHandle, other.tableHandle) 75 | && Objects.equals(constraintSummary, other.constraintSummary) 76 | && Objects.equals(desiredColumns, other.desiredColumns); 77 | } 78 | 79 | @Override 80 | public int hashCode() { 81 | return Objects.hash(tableHandle, 82 | constraintSummary, 83 | desiredColumns); 84 | } 85 | 86 | @Override 87 | public String toString() { 88 | return toStringHelper(this) 89 | .add("tableHandle", tableHandle) 90 | .add("constraintSummary", constraintSummary) 91 | .add("desiredColumns", desiredColumns) 92 | .toString(); 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduRecordSet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.RecordCursor; 22 | import com.facebook.presto.spi.RecordSet; 23 | import com.facebook.presto.spi.type.Type; 24 | import org.apache.kudu.client.KuduScanner; 25 | import org.apache.kudu.client.KuduTable; 26 | 27 | import java.util.HashMap; 28 | import java.util.List; 29 | import java.util.Map; 30 | 31 | import static com.google.common.collect.ImmutableList.toImmutableList; 32 | 33 | public class KuduRecordSet implements RecordSet { 34 | 35 | private final KuduClientSession clientSession; 36 | private final KuduSplit kuduSplit; 37 | private final List columns; 38 | private final boolean containsVirtualRowId; 39 | 40 | public KuduRecordSet(KuduClientSession clientSession, KuduSplit kuduSplit, List columns) { 41 | this.clientSession = clientSession; 42 | this.kuduSplit = kuduSplit; 43 | this.columns = columns; 44 | this.containsVirtualRowId = columns.contains(KuduColumnHandle.ROW_ID_HANDLE); 45 | } 46 | 47 | @Override 48 | public List getColumnTypes() { 49 | return columns.stream() 50 | .map(column -> ((KuduColumnHandle) column).getType()) 51 | .collect(toImmutableList()); 52 | } 53 | 54 | @Override 55 | public RecordCursor cursor() { 56 | KuduScanner scanner = clientSession.createScanner(kuduSplit); 57 | if (!containsVirtualRowId) { 58 | return new KuduRecordCursor(scanner, getColumnTypes()); 59 | } else { 60 | final int primaryKeyColumnCount = kuduSplit.getPrimaryKeyColumnCount(); 61 | 62 | Map fieldMapping = new HashMap<>(); 63 | int index = primaryKeyColumnCount; 64 | for (int i = 0; i < columns.size(); i++) { 65 | KuduColumnHandle handle = (KuduColumnHandle) columns.get(i); 66 | if (!handle.isVirtualRowId()) { 67 | if (handle.getOrdinalPosition() < primaryKeyColumnCount) { 68 | fieldMapping.put(i, handle.getOrdinalPosition()); 69 | } else { 70 | fieldMapping.put(i, index); 71 | index++; 72 | } 73 | } else { 74 | fieldMapping.put(i, -1); 75 | } 76 | } 77 | 78 | KuduTable table = getTable(); 79 | return new KuduRecordCursorWithVirtualRowId(scanner, table, getColumnTypes(), fieldMapping); 80 | } 81 | } 82 | 83 | KuduTable getTable() { 84 | return kuduSplit.getTableHandle().getTable(clientSession); 85 | } 86 | 87 | KuduClientSession getClientSession() { 88 | return clientSession; 89 | } 90 | } 91 | 92 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestIntegerColumnsTest.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.testing.MaterializedResult; 4 | import com.facebook.presto.testing.QueryRunner; 5 | import com.facebook.presto.tests.AbstractTestQueryFramework; 6 | import org.testng.Assert; 7 | import org.testng.annotations.AfterClass; 8 | import org.testng.annotations.BeforeClass; 9 | import org.testng.annotations.Test; 10 | 11 | class TestInt { 12 | final String type; 13 | final int bits; 14 | 15 | TestInt(String type, int bits) { 16 | this.type = type; 17 | this.bits = bits; 18 | } 19 | } 20 | 21 | public class TestIntegerColumnsTest extends AbstractTestQueryFramework { 22 | private QueryRunner queryRunner; 23 | 24 | static final TestInt[] testList = { 25 | new TestInt("TINYINT", 8), 26 | new TestInt("SMALLINT", 16), 27 | new TestInt("INTEGER", 32), 28 | new TestInt("BIGINT", 64), 29 | }; 30 | 31 | 32 | public TestIntegerColumnsTest() throws Exception { 33 | super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("test_integer")); 34 | } 35 | 36 | @Test 37 | public void testCreateTableWithIntegerColumn() { 38 | for (TestInt test : testList) { 39 | doTestCreateTableWithIntegerColumn(test); 40 | } 41 | } 42 | 43 | 44 | public void doTestCreateTableWithIntegerColumn(TestInt test) { 45 | String dropTable = "DROP TABLE IF EXISTS test_int"; 46 | String createTable = "CREATE TABLE test_int (\n"; 47 | createTable += " id INT,\n"; 48 | createTable += " intcol " + test.type + "\n"; 49 | createTable += ") WITH (\n" + 50 | " column_design = '{\"id\": {\"key\": true}}',\n" + 51 | " num_replicas = 1\n" + 52 | ")"; 53 | 54 | queryRunner.execute(dropTable); 55 | queryRunner.execute(createTable); 56 | 57 | long maxValue = Long.MAX_VALUE; 58 | long casted = maxValue >> (64 - test.bits); 59 | queryRunner.execute("INSERT INTO test_int VALUES(1, CAST(" + casted + " AS " + test.type + "))"); 60 | 61 | MaterializedResult result = queryRunner.execute("SELECT id, intcol FROM test_int"); 62 | Assert.assertEquals(result.getRowCount(), 1); 63 | Object obj = result.getMaterializedRows().get(0).getField(1); 64 | switch (test.bits) { 65 | case 64: 66 | Assert.assertTrue(obj instanceof Long); 67 | Assert.assertEquals(((Long) obj).longValue(), casted); 68 | break; 69 | case 32: 70 | Assert.assertTrue(obj instanceof Integer); 71 | Assert.assertEquals(((Integer) obj).longValue(), casted); 72 | break; 73 | case 16: 74 | Assert.assertTrue(obj instanceof Short); 75 | Assert.assertEquals(((Short) obj).longValue(), casted); 76 | break; 77 | case 8: 78 | Assert.assertTrue(obj instanceof Byte); 79 | Assert.assertEquals(((Byte) obj).longValue(), casted); 80 | break; 81 | default: 82 | Assert.fail("Unexpected bits: " + test.bits); 83 | break; 84 | } 85 | } 86 | 87 | @BeforeClass 88 | public void setUp() { 89 | queryRunner = getQueryRunner(); 90 | } 91 | 92 | @AfterClass(alwaysRun = true) 93 | public final void destroy() { 94 | if (queryRunner != null) { 95 | queryRunner.close(); 96 | queryRunner = null; 97 | } 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduColumnHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ColumnMetadata; 22 | import com.facebook.presto.spi.type.Type; 23 | import com.facebook.presto.spi.type.VarbinaryType; 24 | import com.fasterxml.jackson.annotation.JsonCreator; 25 | import com.fasterxml.jackson.annotation.JsonProperty; 26 | import com.google.common.base.MoreObjects; 27 | 28 | import java.util.Objects; 29 | 30 | import static com.google.common.base.MoreObjects.toStringHelper; 31 | import static java.util.Objects.requireNonNull; 32 | 33 | public class KuduColumnHandle implements ColumnHandle { 34 | public static final String ROW_ID = "row_uuid"; 35 | 36 | public static final KuduColumnHandle ROW_ID_HANDLE = new KuduColumnHandle(ROW_ID, -1, VarbinaryType.VARBINARY); 37 | 38 | private final String name; 39 | private final int ordinalPosition; 40 | private final Type type; 41 | 42 | @JsonCreator 43 | public KuduColumnHandle( 44 | @JsonProperty("name") String name, 45 | @JsonProperty("ordinalPosition") int ordinalPosition, 46 | @JsonProperty("type") Type type) { 47 | this.name = requireNonNull(name, "name is null"); 48 | this.ordinalPosition = ordinalPosition; 49 | this.type = requireNonNull(type, "type is null"); 50 | } 51 | 52 | @JsonProperty 53 | public String getName() { 54 | return name; 55 | } 56 | 57 | @JsonProperty 58 | public int getOrdinalPosition() { 59 | return ordinalPosition; 60 | } 61 | 62 | @JsonProperty 63 | public Type getType() { 64 | return type; 65 | } 66 | 67 | 68 | public ColumnMetadata getColumnMetadata() { 69 | return new ColumnMetadata(name, type); 70 | } 71 | 72 | public boolean isVirtualRowId() { 73 | return name.equals(ROW_ID); 74 | } 75 | 76 | @Override 77 | public int hashCode() { 78 | return Objects.hash( 79 | name, 80 | ordinalPosition, 81 | type); 82 | } 83 | 84 | @Override 85 | public boolean equals(Object obj) { 86 | if (this == obj) { 87 | return true; 88 | } 89 | if (obj == null || getClass() != obj.getClass()) { 90 | return false; 91 | } 92 | KuduColumnHandle other = (KuduColumnHandle) obj; 93 | return Objects.equals(this.name, other.name) && 94 | Objects.equals(this.ordinalPosition, other.ordinalPosition) && 95 | Objects.equals(this.type, other.type); 96 | } 97 | 98 | @Override 99 | public String toString() { 100 | MoreObjects.ToStringHelper helper = toStringHelper(this) 101 | .add("name", name) 102 | .add("ordinalPosition", ordinalPosition) 103 | .add("type", type); 104 | 105 | return helper.toString(); 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/KuduQueryRunnerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.Session; 17 | import com.facebook.presto.testing.QueryRunner; 18 | import com.facebook.presto.tests.DistributedQueryRunner; 19 | import com.facebook.presto.tpch.TpchPlugin; 20 | import com.google.common.collect.ImmutableList; 21 | import com.google.common.collect.ImmutableMap; 22 | import io.airlift.tpch.TpchTable; 23 | 24 | import java.util.Map; 25 | 26 | import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; 27 | import static com.facebook.presto.testing.TestingSession.testSessionBuilder; 28 | import static com.facebook.presto.tests.QueryAssertions.copyTpchTables; 29 | import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; 30 | import static io.airlift.testing.Closeables.closeAllSuppress; 31 | import static java.util.Locale.ENGLISH; 32 | 33 | public class KuduQueryRunnerFactory 34 | { 35 | private KuduQueryRunnerFactory() {} 36 | 37 | public static QueryRunner createKuduQueryRunner(String schema) 38 | throws Exception 39 | { 40 | QueryRunner runner = null; 41 | String kuduSchema = schema; 42 | try { 43 | runner = DistributedQueryRunner.builder(createSession(kuduSchema)).setNodeCount(3).build(); 44 | 45 | installKuduConnector(runner, kuduSchema); 46 | 47 | return runner; 48 | } 49 | catch (Throwable e) { 50 | closeAllSuppress(e, runner); 51 | throw e; 52 | } 53 | } 54 | 55 | public static QueryRunner createKuduQueryRunnerTpch(TpchTable... tables) 56 | throws Exception 57 | { 58 | return createKuduQueryRunnerTpch(ImmutableList.copyOf(tables)); 59 | } 60 | 61 | public static QueryRunner createKuduQueryRunnerTpch(Iterable> tables) 62 | throws Exception 63 | { 64 | DistributedQueryRunner runner = null; 65 | String kuduSchema = "tpch"; 66 | try { 67 | runner = DistributedQueryRunner.builder(createSession(kuduSchema)).setNodeCount(3).build(); 68 | 69 | runner.installPlugin(new TpchPlugin()); 70 | runner.createCatalog("tpch", "tpch"); 71 | 72 | installKuduConnector(runner, kuduSchema); 73 | 74 | copyTpchTables(runner, "tpch", TINY_SCHEMA_NAME, createSession(kuduSchema), tables); 75 | 76 | return runner; 77 | } 78 | catch (Throwable e) { 79 | closeAllSuppress(e, runner); 80 | throw e; 81 | } 82 | } 83 | 84 | private static void installKuduConnector(QueryRunner runner, String schema) 85 | { 86 | String masterAddresses = System.getProperty("kudu.client.master-addresses", "localhost:7051"); 87 | Map properties = ImmutableMap.of( 88 | "kudu.client.master-addresses", masterAddresses); 89 | 90 | runner.installPlugin(new KuduPlugin()); 91 | runner.createCatalog("kudu", "kudu", properties); 92 | 93 | runner.execute("DROP SCHEMA IF EXISTS " + schema); 94 | runner.execute("CREATE SCHEMA " + schema); 95 | } 96 | 97 | public static Session createSession(String schema) 98 | { 99 | return testSessionBuilder() 100 | .setCatalog("kudu") 101 | .setSchema(schema) 102 | .setTimeZoneKey(UTC_KEY) 103 | .setLocale(ENGLISH) 104 | .build(); 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduClientConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.google.common.base.Splitter; 21 | import io.airlift.configuration.Config; 22 | import io.airlift.units.Duration; 23 | import io.airlift.units.MaxDuration; 24 | import io.airlift.units.MinDuration; 25 | 26 | import java.util.Arrays; 27 | import java.util.List; 28 | import java.util.concurrent.TimeUnit; 29 | 30 | import javax.validation.constraints.NotNull; 31 | import javax.validation.constraints.Size; 32 | 33 | /** 34 | * Configuration read from etc/catalog/kudu.properties 35 | */ 36 | public class KuduClientConfig { 37 | private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); 38 | 39 | private List masterAddresses; 40 | private Duration defaultAdminOperationTimeout = new Duration(30, TimeUnit.SECONDS); 41 | private Duration defaultOperationTimeout = new Duration(30, TimeUnit.SECONDS); 42 | private Duration defaultSocketReadTimeout = new Duration(10, TimeUnit.SECONDS); 43 | private boolean disableStatistics = false; 44 | private String tenant = null; 45 | 46 | @NotNull 47 | @Size(min = 1) 48 | public List getMasterAddresses() 49 | { 50 | return masterAddresses; 51 | } 52 | 53 | @Config("kudu.client.master-addresses") 54 | public KuduClientConfig setMasterAddresses(String commaSeparatedList) 55 | { 56 | this.masterAddresses = SPLITTER.splitToList(commaSeparatedList); 57 | return this; 58 | } 59 | 60 | public KuduClientConfig setMasterAddresses(String... contactPoints) 61 | { 62 | this.masterAddresses = Arrays.asList(contactPoints); 63 | return this; 64 | } 65 | 66 | @Config("kudu.client.defaultAdminOperationTimeout") 67 | public KuduClientConfig setDefaultAdminOperationTimeout(Duration timeout) 68 | { 69 | this.defaultAdminOperationTimeout = timeout; 70 | return this; 71 | } 72 | 73 | @MinDuration("1s") 74 | @MaxDuration("1h") 75 | public Duration getDefaultAdminOperationTimeout() 76 | { 77 | return defaultAdminOperationTimeout; 78 | } 79 | 80 | @Config("kudu.client.defaultOperationTimeout") 81 | public KuduClientConfig setDefaultOperationTimeout(Duration timeout) 82 | { 83 | this.defaultOperationTimeout = timeout; 84 | return this; 85 | } 86 | 87 | @MinDuration("1s") 88 | @MaxDuration("1h") 89 | public Duration getDefaultOperationTimeout() 90 | { 91 | return defaultOperationTimeout; 92 | } 93 | 94 | @Config("kudu.client.defaultSocketReadTimeout") 95 | public KuduClientConfig setDefaultSocketReadTimeout(Duration timeout) 96 | { 97 | this.defaultSocketReadTimeout = timeout; 98 | return this; 99 | } 100 | 101 | @MinDuration("1s") 102 | @MaxDuration("1h") 103 | public Duration getDefaultSocketReadTimeout() 104 | { 105 | return defaultSocketReadTimeout; 106 | } 107 | 108 | public boolean isDisableStatistics() 109 | { 110 | return this.disableStatistics; 111 | } 112 | 113 | @Config("kudu.client.disableStatistics") 114 | public KuduClientConfig setDisableStatistics(boolean disableStatistics) 115 | { 116 | this.disableStatistics = disableStatistics; 117 | return this; 118 | } 119 | 120 | public String getTenant() 121 | { 122 | return tenant; 123 | } 124 | 125 | @Config("kudu.session.tenant") 126 | public KuduClientConfig setTenant(String tenant) 127 | { 128 | this.tenant = tenant; 129 | return this; 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduModule.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.connector.ConnectorPageSinkProvider; 21 | import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; 22 | import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; 23 | import com.facebook.presto.spi.connector.ConnectorSplitManager; 24 | import com.facebook.presto.spi.procedure.Procedure; 25 | import com.facebook.presto.spi.type.TypeManager; 26 | import com.google.inject.AbstractModule; 27 | import com.google.inject.Provides; 28 | import com.google.inject.Scopes; 29 | import com.google.inject.multibindings.Multibinder; 30 | import com.google.inject.multibindings.ProvidesIntoSet; 31 | import ml.littlebulb.presto.kudu.procedures.RangePartitionProcedures; 32 | import ml.littlebulb.presto.kudu.properties.KuduTableProperties; 33 | import org.apache.kudu.client.KuduClient; 34 | 35 | import javax.inject.Singleton; 36 | 37 | import static io.airlift.configuration.ConfigBinder.configBinder; 38 | import static java.util.Objects.requireNonNull; 39 | 40 | public class KuduModule extends AbstractModule { 41 | 42 | private final String connectorId; 43 | private final TypeManager typeManager; 44 | 45 | public KuduModule(String connectorId, TypeManager typeManager) { 46 | this.connectorId = requireNonNull(connectorId, "connector id is null"); 47 | this.typeManager = requireNonNull(typeManager, "typeManager is null"); 48 | } 49 | 50 | @Override 51 | protected void configure() { 52 | bind(TypeManager.class).toInstance(typeManager); 53 | 54 | bind(KuduConnector.class).in(Scopes.SINGLETON); 55 | bind(KuduConnectorId.class).toInstance(new KuduConnectorId(connectorId)); 56 | bind(KuduMetadata.class).in(Scopes.SINGLETON); 57 | bind(KuduTableProperties.class).in(Scopes.SINGLETON); 58 | bind(ConnectorSplitManager.class).to(KuduSplitManager.class).in(Scopes.SINGLETON); 59 | bind(ConnectorRecordSetProvider.class).to(KuduRecordSetProvider.class) 60 | .in(Scopes.SINGLETON); 61 | bind(ConnectorPageSourceProvider.class).to(KuduPageSourceProvider.class) 62 | .in(Scopes.SINGLETON); 63 | bind(ConnectorPageSinkProvider.class).to(KuduPageSinkProvider.class).in(Scopes.SINGLETON); 64 | bind(KuduHandleResolver.class).in(Scopes.SINGLETON); 65 | bind(KuduRecordSetProvider.class).in(Scopes.SINGLETON); 66 | configBinder(binder()).bindConfig(KuduClientConfig.class); 67 | 68 | bind(RangePartitionProcedures.class).in(Scopes.SINGLETON); 69 | Multibinder.newSetBinder(binder(), Procedure.class); 70 | } 71 | 72 | @ProvidesIntoSet 73 | Procedure getAddRangePartitionProcedure(RangePartitionProcedures procedures) { 74 | return procedures.getAddPartitionProcedure(); 75 | } 76 | 77 | @ProvidesIntoSet 78 | Procedure getDropRangePartitionProcedure(RangePartitionProcedures procedures) { 79 | return procedures.getDropPartitionProcedure(); 80 | } 81 | 82 | @Singleton 83 | @Provides 84 | KuduClientSession createKuduClientSession( 85 | KuduConnectorId connectorId, 86 | KuduClientConfig config) { 87 | requireNonNull(config, "config is null"); 88 | 89 | KuduClient.KuduClientBuilder builder = new KuduClient.KuduClientBuilder(config.getMasterAddresses()); 90 | builder.defaultAdminOperationTimeoutMs(config.getDefaultAdminOperationTimeout().toMillis()); 91 | builder.defaultOperationTimeoutMs(config.getDefaultOperationTimeout().toMillis()); 92 | builder.defaultSocketReadTimeoutMs(config.getDefaultSocketReadTimeout().toMillis()); 93 | if (config.isDisableStatistics()) { 94 | builder.disableStatistics(); 95 | } 96 | KuduClient client = builder.build(); 97 | String tenant = config.getTenant(); 98 | return new NativeKuduClientSession(connectorId, client, tenant); 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduConnector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.connector.Connector; 21 | import com.facebook.presto.spi.connector.ConnectorMetadata; 22 | import com.facebook.presto.spi.connector.ConnectorPageSinkProvider; 23 | import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; 24 | import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; 25 | import com.facebook.presto.spi.connector.ConnectorSplitManager; 26 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle; 27 | import com.facebook.presto.spi.procedure.Procedure; 28 | import com.facebook.presto.spi.session.PropertyMetadata; 29 | import com.facebook.presto.spi.transaction.IsolationLevel; 30 | import com.google.common.collect.ImmutableSet; 31 | import io.airlift.bootstrap.LifeCycleManager; 32 | import io.airlift.log.Logger; 33 | import ml.littlebulb.presto.kudu.properties.KuduTableProperties; 34 | 35 | import java.util.List; 36 | import java.util.Set; 37 | 38 | import javax.inject.Inject; 39 | 40 | import static com.facebook.presto.spi.transaction.IsolationLevel.READ_COMMITTED; 41 | import static com.facebook.presto.spi.transaction.IsolationLevel.checkConnectorSupports; 42 | import static java.util.Objects.requireNonNull; 43 | 44 | public class KuduConnector implements Connector { 45 | 46 | private static final Logger log = Logger.get(KuduConnector.class); 47 | 48 | private final LifeCycleManager lifeCycleManager; 49 | private final KuduMetadata metadata; 50 | private final ConnectorSplitManager splitManager; 51 | private final ConnectorRecordSetProvider recordSetProvider; 52 | private final ConnectorPageSourceProvider pageSourceProvider; 53 | private final KuduTableProperties tableProperties; 54 | private final ConnectorPageSinkProvider pageSinkProvider; 55 | private final Set procedures; 56 | 57 | @Inject 58 | public KuduConnector(LifeCycleManager lifeCycleManager, KuduMetadata metadata, 59 | ConnectorSplitManager splitManager, ConnectorRecordSetProvider recordSetProvider, 60 | KuduTableProperties tableProperties, 61 | ConnectorPageSourceProvider pageSourceProvider, 62 | ConnectorPageSinkProvider pageSinkProvider, 63 | Set procedures) { 64 | this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); 65 | this.metadata = requireNonNull(metadata, "metadata is null"); 66 | this.splitManager = requireNonNull(splitManager, "splitManager is null"); 67 | this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); 68 | this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); 69 | this.tableProperties = requireNonNull(tableProperties, "tableProperties is null"); 70 | this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null"); 71 | this.procedures = ImmutableSet.copyOf(requireNonNull(procedures, "procedures is null")); 72 | } 73 | 74 | @Override 75 | public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, 76 | boolean readOnly) { 77 | checkConnectorSupports(READ_COMMITTED, isolationLevel); 78 | return KuduTransactionHandle.INSTANCE; 79 | } 80 | 81 | @Override 82 | public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) { 83 | return metadata; 84 | } 85 | 86 | @Override 87 | public ConnectorSplitManager getSplitManager() { 88 | return splitManager; 89 | } 90 | 91 | @Override 92 | public ConnectorRecordSetProvider getRecordSetProvider() { 93 | return recordSetProvider; 94 | } 95 | 96 | @Override 97 | public ConnectorPageSourceProvider getPageSourceProvider() { 98 | return pageSourceProvider; 99 | } 100 | 101 | @Override 102 | public ConnectorPageSinkProvider getPageSinkProvider() { 103 | return pageSinkProvider; 104 | } 105 | 106 | @Override 107 | public List> getTableProperties() { 108 | return tableProperties.getTableProperties(); 109 | } 110 | 111 | @Override 112 | public Set getProcedures() { 113 | return procedures; 114 | } 115 | 116 | @Override 117 | public final void shutdown() { 118 | try { 119 | lifeCycleManager.stop(); 120 | } catch (Exception e) { 121 | log.error(e, "Error shutting down connector"); 122 | } 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduRecordCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.RecordCursor; 21 | import com.facebook.presto.spi.type.Type; 22 | import io.airlift.log.Logger; 23 | import io.airlift.slice.Slice; 24 | import org.apache.kudu.client.KuduException; 25 | import org.apache.kudu.client.KuduScanner; 26 | import org.apache.kudu.client.RowResult; 27 | import org.apache.kudu.client.RowResultIterator; 28 | 29 | import java.lang.reflect.Field; 30 | import java.util.List; 31 | 32 | public class KuduRecordCursor implements RecordCursor { 33 | 34 | private static final Logger log = Logger.get(KuduRecordCursor.class); 35 | 36 | private final KuduScanner scanner; 37 | private final List columnTypes; 38 | private final Field rowDataField; 39 | private RowResultIterator nextRows; 40 | protected RowResult currentRow; 41 | 42 | private long totalBytes; 43 | private long nanoStart; 44 | private long nanoEnd; 45 | private boolean started; 46 | 47 | public KuduRecordCursor(KuduScanner scanner, List columnTypes) { 48 | this.scanner = scanner; 49 | this.columnTypes = columnTypes; 50 | Field field = null; 51 | try { 52 | field = RowResult.class.getDeclaredField("rawData"); 53 | field.setAccessible(true); 54 | } catch (NoSuchFieldException e) { 55 | // ignore 56 | } 57 | this.rowDataField = field; 58 | } 59 | 60 | @Override 61 | public long getCompletedBytes() { 62 | return totalBytes; 63 | } 64 | 65 | @Override 66 | public long getReadTimeNanos() { 67 | return nanoStart > 0L ? (nanoEnd == 0 ? System.nanoTime() : nanoEnd) - nanoStart : 0L; 68 | } 69 | 70 | @Override 71 | public Type getType(int field) { 72 | return columnTypes.get(field); 73 | } 74 | 75 | protected int mapping(int field) { return field; } 76 | 77 | /** 78 | * get next Row/Page 79 | */ 80 | @Override 81 | public boolean advanceNextPosition() { 82 | boolean needNextRows = !started || !nextRows.hasNext(); 83 | 84 | if (!started) { 85 | started = true; 86 | nanoStart = System.nanoTime(); 87 | } 88 | 89 | if (needNextRows) { 90 | currentRow = null; 91 | try { 92 | do { 93 | if (!scanner.hasMoreRows()) { 94 | return false; 95 | } 96 | 97 | nextRows = scanner.nextRows(); 98 | } while (!nextRows.hasNext()); 99 | log.debug("Fetched " + nextRows.getNumRows() + " rows"); 100 | } catch (KuduException e) { 101 | currentRow = null; 102 | throw new RuntimeException(e); 103 | } 104 | } 105 | 106 | currentRow = nextRows.next(); 107 | totalBytes += getRowLength(); 108 | return true; 109 | } 110 | 111 | private org.apache.kudu.util.Slice getCurrentRowRawData() { 112 | if (rowDataField != null && currentRow != null) { 113 | try { 114 | return ((org.apache.kudu.util.Slice) rowDataField.get(currentRow)); 115 | } catch (IllegalAccessException e) { 116 | return null; 117 | } 118 | } else { 119 | return null; 120 | } 121 | } 122 | 123 | private int getRowLength() { 124 | org.apache.kudu.util.Slice rawData = getCurrentRowRawData(); 125 | if (rawData != null) { 126 | return rawData.length(); 127 | } else { 128 | return columnTypes.size(); 129 | } 130 | } 131 | 132 | @Override 133 | public boolean getBoolean(int field) { 134 | int index = mapping(field); 135 | return TypeHelper.getBoolean(columnTypes.get(field), currentRow, index); 136 | } 137 | 138 | @Override 139 | public long getLong(int field) { 140 | int index = mapping(field); 141 | return TypeHelper.getLong(columnTypes.get(field), currentRow, index); 142 | } 143 | 144 | @Override 145 | public double getDouble(int field) { 146 | int index = mapping(field); 147 | return TypeHelper.getDouble(columnTypes.get(field), currentRow, index); 148 | } 149 | 150 | @Override 151 | public Slice getSlice(int field) { 152 | int index = mapping(field); 153 | return TypeHelper.getSlice(columnTypes.get(field), currentRow, index); 154 | } 155 | 156 | @Override 157 | public Object getObject(int field) { 158 | int index = mapping(field); 159 | return TypeHelper.getObject(columnTypes.get(field), currentRow, index); 160 | } 161 | 162 | @Override 163 | public boolean isNull(int field) { 164 | int mappedField = mapping(field); 165 | return mappedField >= 0 && currentRow.isNull(mappedField); 166 | } 167 | 168 | @Override 169 | public void close() { 170 | nanoEnd = System.nanoTime(); 171 | currentRow = null; 172 | nextRows = null; 173 | } 174 | } 175 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduPageSink.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.spi.ConnectorPageSink; 4 | import com.facebook.presto.spi.ConnectorSession; 5 | import com.facebook.presto.spi.Page; 6 | import com.facebook.presto.spi.block.Block; 7 | import com.facebook.presto.spi.type.DecimalType; 8 | import com.facebook.presto.spi.type.SqlDate; 9 | import com.facebook.presto.spi.type.SqlDecimal; 10 | import com.facebook.presto.spi.type.Type; 11 | import com.google.common.base.Charsets; 12 | import com.google.common.collect.ImmutableList; 13 | import io.airlift.slice.Slice; 14 | import org.apache.kudu.client.KuduException; 15 | import org.apache.kudu.client.KuduSession; 16 | import org.apache.kudu.client.KuduTable; 17 | import org.apache.kudu.client.PartialRow; 18 | import org.apache.kudu.client.SessionConfiguration; 19 | import org.apache.kudu.client.Upsert; 20 | 21 | import java.time.LocalDateTime; 22 | import java.time.ZoneOffset; 23 | import java.time.format.DateTimeFormatter; 24 | import java.util.Collection; 25 | import java.util.List; 26 | import java.util.UUID; 27 | import java.util.concurrent.CompletableFuture; 28 | import java.util.concurrent.TimeUnit; 29 | 30 | import static com.facebook.presto.spi.type.BigintType.BIGINT; 31 | import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; 32 | import static com.facebook.presto.spi.type.DateType.DATE; 33 | import static com.facebook.presto.spi.type.DoubleType.DOUBLE; 34 | import static com.facebook.presto.spi.type.IntegerType.INTEGER; 35 | import static com.facebook.presto.spi.type.RealType.REAL; 36 | import static com.facebook.presto.spi.type.SmallintType.SMALLINT; 37 | import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; 38 | import static com.facebook.presto.spi.type.TinyintType.TINYINT; 39 | import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; 40 | import static com.facebook.presto.spi.type.Varchars.isVarcharType; 41 | import static java.lang.Float.intBitsToFloat; 42 | import static java.util.Objects.requireNonNull; 43 | import static java.util.concurrent.CompletableFuture.completedFuture; 44 | 45 | public class KuduPageSink implements ConnectorPageSink { 46 | private final ConnectorSession connectorSession; 47 | private final KuduSession session; 48 | private final KuduTable table; 49 | private final List columnTypes; 50 | private final List originalColumnTypes; 51 | private final boolean generateUUID; 52 | 53 | private final String uuid; 54 | private int nextSubId = 0; 55 | 56 | public KuduPageSink(ConnectorSession connectorSession, KuduClientSession clientSession, 57 | KuduExtendedTableHandle extendedTableHandle, 58 | boolean generateUUID) { 59 | requireNonNull(clientSession, "clientSession is null"); 60 | this.connectorSession = connectorSession; 61 | this.columnTypes = extendedTableHandle.getColumnTypes(); 62 | this.originalColumnTypes = extendedTableHandle.getOriginalColumnTypes(); 63 | this.generateUUID = generateUUID; 64 | 65 | this.table = extendedTableHandle.getTable(clientSession); 66 | this.session = clientSession.newSession(); 67 | this.session.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLUSH_BACKGROUND); 68 | uuid = UUID.randomUUID().toString(); 69 | } 70 | 71 | @Override 72 | public CompletableFuture appendPage(Page page) { 73 | for (int position = 0; position < page.getPositionCount(); position++) { 74 | Upsert upsert = table.newUpsert(); 75 | PartialRow row = upsert.getRow(); 76 | int start = 0; 77 | if (generateUUID) { 78 | String id = String.format("%s-%08x", uuid, nextSubId++); 79 | row.addString(0, id); 80 | start = 1; 81 | } 82 | 83 | for (int channel = 0; channel < page.getChannelCount(); channel++) { 84 | appendColumn(row, page, position, channel, channel + start); 85 | } 86 | 87 | try { 88 | session.apply(upsert); 89 | } catch (KuduException e) { 90 | throw new RuntimeException(e); 91 | } 92 | } 93 | return NOT_BLOCKED; 94 | } 95 | 96 | private void appendColumn(PartialRow row, Page page, int position, int channel, int destChannel) { 97 | Block block = page.getBlock(channel); 98 | Type type = columnTypes.get(destChannel); 99 | if (block.isNull(position)) { 100 | row.setNull(destChannel); 101 | } else if (TIMESTAMP.equals(type)) { 102 | row.addLong(destChannel, type.getLong(block, position) * 1000); 103 | } else if (REAL.equals(type)) { 104 | row.addFloat(destChannel, intBitsToFloat((int) type.getLong(block, position))); 105 | } else if (BIGINT.equals(type)) { 106 | row.addLong(destChannel, type.getLong(block, position)); 107 | } else if (INTEGER.equals(type)) { 108 | row.addInt(destChannel, (int) type.getLong(block, position)); 109 | } else if (SMALLINT.equals(type)) { 110 | row.addShort(destChannel, (short) type.getLong(block, position)); 111 | } else if (TINYINT.equals(type)) { 112 | row.addByte(destChannel, (byte) type.getLong(block, position)); 113 | } else if (BOOLEAN.equals(type)) { 114 | row.addBoolean(destChannel, type.getBoolean(block, position)); 115 | } else if (DOUBLE.equals(type)) { 116 | row.addDouble(destChannel, type.getDouble(block, position)); 117 | } else if (isVarcharType(type)) { 118 | Type originalType = originalColumnTypes.get(destChannel); 119 | if (DATE.equals(originalType)) { 120 | SqlDate date = (SqlDate) originalType.getObjectValue(connectorSession, block, position); 121 | LocalDateTime ldt = LocalDateTime.ofEpochSecond(TimeUnit.DAYS.toSeconds(date.getDays()), 0, ZoneOffset.UTC); 122 | byte[] bytes = ldt.format(DateTimeFormatter.ISO_LOCAL_DATE).getBytes(Charsets.UTF_8); 123 | row.addStringUtf8(destChannel, bytes); 124 | } else { 125 | row.addString(destChannel, type.getSlice(block, position).toStringUtf8()); 126 | } 127 | } else if (VARBINARY.equals(type)) { 128 | row.addBinary(destChannel, type.getSlice(block, position).toByteBuffer()); 129 | } else if (type instanceof DecimalType) { 130 | SqlDecimal sqlDecimal = (SqlDecimal) type.getObjectValue(connectorSession, block, position); 131 | row.addDecimal(destChannel, sqlDecimal.toBigDecimal()); 132 | } else { 133 | throw new UnsupportedOperationException("Type is not supported: " + type); 134 | } 135 | } 136 | 137 | @Override 138 | public CompletableFuture> finish() { 139 | closeSession(); 140 | // the committer does not need any additional info 141 | return completedFuture(ImmutableList.of()); 142 | } 143 | 144 | @Override 145 | public void abort() { 146 | closeSession(); 147 | } 148 | 149 | private void closeSession() { 150 | try { 151 | session.close(); 152 | } catch (KuduException e) { 153 | throw new RuntimeException(e); 154 | } 155 | } 156 | } 157 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 4.0.0 21 | 22 | ml.littlebulb.presto.kudu 23 | presto-kudu 24 | Kudu Connector for Presto 25 | 0.208.3-SNAPSHOT 26 | presto-plugin 27 | 28 | 29 | 0.208 30 | 1.8.0 31 | 0.172 32 | 6.10 33 | 0.36 34 | 4.2.0 35 | 24.1-jre 36 | 37 | 38 | 39 | 40 | org.apache.kudu 41 | kudu-client 42 | ${kudu.version} 43 | 44 | 45 | 46 | io.airlift 47 | bootstrap 48 | ${dep.airlift.version} 49 | 50 | 51 | org.slf4j 52 | log4j-over-slf4j 53 | 54 | 55 | 56 | 57 | 58 | io.airlift 59 | json 60 | ${dep.airlift.version} 61 | 62 | 63 | 64 | io.airlift 65 | log 66 | ${dep.airlift.version} 67 | 68 | 69 | 70 | com.google.guava 71 | guava 72 | ${dep.guava.version} 73 | 74 | 75 | 76 | com.google.inject 77 | guice 78 | ${dep.guice.version} 79 | 80 | 81 | 82 | com.google.inject.extensions 83 | guice-multibindings 84 | ${dep.guice.version} 85 | 86 | 87 | 88 | 89 | com.facebook.presto 90 | presto-spi 91 | ${presto.version} 92 | provided 93 | 94 | 95 | 96 | io.airlift 97 | slice 98 | ${dep.slice.version} 99 | provided 100 | 101 | 102 | 103 | io.airlift 104 | units 105 | 1.0 106 | provided 107 | 108 | 109 | 110 | com.fasterxml.jackson.core 111 | jackson-annotations 112 | 2.8.9 113 | provided 114 | 115 | 116 | 117 | 118 | org.testng 119 | testng 120 | ${dep.testng.version} 121 | test 122 | 123 | 124 | 125 | io.airlift 126 | testing 127 | ${dep.airlift.version} 128 | test 129 | 130 | 131 | 132 | com.facebook.presto 133 | presto-main 134 | ${presto.version} 135 | test 136 | 137 | 138 | 139 | com.facebook.presto 140 | presto-tests 141 | ${presto.version} 142 | test 143 | 144 | 145 | 146 | 147 | 148 | 149 | 150 | maven-compiler-plugin 151 | 152 | 1.8 153 | 1.8 154 | 155 | 156 | 157 | org.apache.maven.plugins 158 | maven-surefire-plugin 159 | 2.20.1 160 | 161 | 162 | 163 | **/Test*.java 164 | **/*Test.java 165 | **/*TestCase.java 166 | **/*Suite.java 167 | 168 | ${project.build.directory}/surefire-reports 169 | -Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m 170 | 171 | true 172 | 173 | false 174 | 175 | 176 | 177 | 178 | org.apache.maven.plugins 179 | maven-checkstyle-plugin 180 | 2.17 181 | 182 | true 183 | 184 | 185 | 186 | 187 | org.apache.maven.plugins 188 | maven-enforcer-plugin 189 | 3.0.0-M1 190 | 191 | true 192 | 193 | 194 | 195 | 196 | com.ning.maven.plugins 197 | maven-dependency-versions-check-plugin 198 | 199 | true 200 | false 201 | 202 | 203 | 204 | 205 | org.apache.maven.plugins 206 | maven-dependency-plugin 207 | 208 | false 209 | 210 | 211 | 212 | 213 | com.ning.maven.plugins 214 | maven-duplicate-finder-plugin 215 | 216 | true 217 | 218 | 219 | 220 | 221 | com.facebook.presto 222 | presto-maven-plugin 223 | 0.3 224 | true 225 | 226 | 227 | 228 | pl.project13.maven 229 | git-commit-id-plugin 230 | 231 | true 232 | 233 | 234 | 235 | 236 | 237 | -------------------------------------------------------------------------------- /src/test/java/ml/littlebulb/presto/kudu/TestRangePartitioningTest.java: -------------------------------------------------------------------------------- 1 | package ml.littlebulb.presto.kudu; 2 | 3 | import com.facebook.presto.testing.MaterializedResult; 4 | import com.facebook.presto.testing.QueryRunner; 5 | import com.facebook.presto.tests.AbstractTestQueryFramework; 6 | import org.testng.Assert; 7 | import org.testng.annotations.AfterClass; 8 | import org.testng.annotations.BeforeClass; 9 | import org.testng.annotations.Test; 10 | 11 | class TestRanges { 12 | final String[] types; 13 | final String range1, range2, range3, range4; 14 | final String cmp1, cmp2, cmp3, cmp4; 15 | 16 | TestRanges(String type, String range1, String range2, String range3, String range4) { 17 | this(new String[]{type}, range1, range2, range3, range4, range1, range2, range3, range4); 18 | } 19 | 20 | TestRanges(String type, String range1, String range2, String range3, String range4, 21 | String cmp1, String cmp2, String cmp3, String cmp4) { 22 | this(new String[]{type}, range1, range2, range3, range4, cmp1, cmp2, cmp3, cmp4); 23 | } 24 | 25 | TestRanges(String[] types, String range1, String range2, String range3, String range4) { 26 | this(types, range1, range2, range3, range4, range1, range2, range3, range4); 27 | } 28 | 29 | TestRanges(String[] types, String range1, String range2, String range3, String range4, 30 | String cmp1, String cmp2, String cmp3, String cmp4) { 31 | this.types = types; 32 | this.range1 = range1; 33 | this.range2 = range2; 34 | this.range3 = range3; 35 | this.range4 = range4; 36 | this.cmp1 = cmp1; 37 | this.cmp2 = cmp2; 38 | this.cmp3 = cmp3; 39 | this.cmp4 = cmp4; 40 | } 41 | } 42 | 43 | public class TestRangePartitioningTest extends AbstractTestQueryFramework { 44 | private QueryRunner queryRunner; 45 | 46 | static final TestRanges[] testRangesList = { 47 | new TestRanges("varchar", 48 | "{\"lower\": null, \"upper\": \"D\"}", 49 | "{\"lower\": \"D\", \"upper\": \"M\"}", 50 | "{\"lower\": \"M\", \"upper\": \"S\"}", 51 | "{\"lower\": \"S\", \"upper\": null}"), 52 | new TestRanges("timestamp", 53 | "{\"lower\": null, \"upper\": \"2017-01-01T02:03:04.567Z\"}", 54 | "{\"lower\": \"2017-01-01 03:03:04.567+01:00\", \"upper\": \"2017-02-01 12:34\"}", 55 | "{\"lower\": \"2017-02-01 12:34\", \"upper\": \"2017-03-01\"}", 56 | "{\"lower\": \"2017-03-01\", \"upper\": null}", 57 | "{\"lower\": null, \"upper\": \"2017-01-01T02:03:04.567Z\"}", 58 | "{\"lower\": \"2017-01-01T02:03:04.567Z\", \"upper\": \"2017-02-01T12:34:00.000Z\"}", 59 | "{\"lower\": \"2017-02-01T12:34:00.000Z\", \"upper\": \"2017-03-01T00:00:00.000Z\"}", 60 | "{\"lower\": \"2017-03-01T00:00:00.000Z\", \"upper\": null}"), 61 | new TestRanges("tinyint", 62 | "{\"lower\": null, \"upper\": -10}", 63 | "{\"lower\": \"-10\", \"upper\": 0}", 64 | "{\"lower\": 0, \"upper\": 10}", 65 | "{\"lower\": 10, \"upper\": 20}", 66 | "{\"lower\": null, \"upper\": -10}", 67 | "{\"lower\": -10, \"upper\": 0}", 68 | "{\"lower\": 0, \"upper\": 10}", 69 | "{\"lower\": 10, \"upper\": 20}"), 70 | new TestRanges("smallint", 71 | "{\"lower\": null, \"upper\": -1000}", 72 | "{\"lower\": -1000, \"upper\": 0}", 73 | "{\"lower\": 0, \"upper\": 1000}", 74 | "{\"lower\": 1000, \"upper\": 2000}"), 75 | new TestRanges("integer", 76 | "{\"lower\": null, \"upper\": -1000000}", 77 | "{\"lower\": -1000000, \"upper\": 0}", 78 | "{\"lower\": 0, \"upper\": 10000}", 79 | "{\"lower\": 10000, \"upper\": 1000000}"), 80 | new TestRanges("bigint", 81 | "{\"lower\": null, \"upper\": \"-123456789012345\"}", 82 | "{\"lower\": \"-123456789012345\", \"upper\": 0}", 83 | "{\"lower\": 0, \"upper\": 123400}", 84 | "{\"lower\": 123400, \"upper\": 123456789012345}", 85 | "{\"lower\": null, \"upper\": -123456789012345}", 86 | "{\"lower\": -123456789012345, \"upper\": 0}", 87 | "{\"lower\": 0, \"upper\": 123400}", 88 | "{\"lower\": 123400, \"upper\": 123456789012345}"), 89 | new TestRanges("varbinary", 90 | "{\"lower\": null, \"upper\": \"YWI=\"}", 91 | "{\"lower\": \"YWI=\", \"upper\": \"ZA==\"}", 92 | "{\"lower\": \"ZA==\", \"upper\": \"bW1t\"}", 93 | "{\"lower\": \"bW1t\", \"upper\": \"eg==\"}"), 94 | new TestRanges(new String[]{"smallint", "varchar"}, 95 | "{\"lower\": null, \"upper\": [1, \"M\"]}", 96 | "{\"lower\": [1, \"M\"], \"upper\": [1, \"T\"]}", 97 | "{\"lower\": [1, \"T\"], \"upper\": [2, \"Z\"]}", 98 | "{\"lower\": [2, \"Z\"], \"upper\": null}"), 99 | }; 100 | 101 | 102 | public TestRangePartitioningTest() { 103 | super(() -> KuduQueryRunnerFactory.createKuduQueryRunner("range_partitioning")); 104 | } 105 | 106 | @Test 107 | public void testCreateAndChangeTableWithRangePartition() { 108 | for (TestRanges ranges : testRangesList) { 109 | doTestCreateAndChangeTableWithRangePartition(ranges); 110 | } 111 | } 112 | 113 | 114 | public void doTestCreateAndChangeTableWithRangePartition(TestRanges ranges) { 115 | String[] types = ranges.types; 116 | String name = String.join("_", ranges.types); 117 | String tableName = "range_partitioning_" + name; 118 | String createTable = "CREATE TABLE " + tableName + " (\n"; 119 | String columnDesign = ""; 120 | String partitionDesign = ""; 121 | for (int i = 0; i < types.length; i++) { 122 | String type = types[i]; 123 | String columnName = "key" + i; 124 | createTable += " " + columnName + " " + type + ",\n"; 125 | if (i == 0) { 126 | columnDesign += "{"; 127 | partitionDesign += "["; 128 | } else { 129 | columnDesign += ","; 130 | partitionDesign += ","; 131 | } 132 | columnDesign += "\"" + columnName + "\": {\"key\": true}"; 133 | partitionDesign += "\"" + columnName + "\""; 134 | } 135 | columnDesign += "}"; 136 | partitionDesign += "]"; 137 | 138 | createTable += 139 | " value varchar\n" + 140 | ") WITH (\n" + 141 | " column_design = '" + columnDesign + "',\n" + 142 | " partition_design = '{\"range\": {\"columns\":" + partitionDesign + "}}',\n" + 143 | " range_partitions = '[" + ranges.range1 + "," + ranges.range2 + "]',\n" + 144 | " num_replicas = 1\n" + 145 | ")"; 146 | queryRunner.execute(createTable); 147 | 148 | String schema = queryRunner.getDefaultSession().getSchema().get(); 149 | String addPartition3 = "CALL kudu.system.add_range_partition('" + schema + "','" + tableName + "','" + ranges.range3 + "')"; 150 | queryRunner.execute(addPartition3); 151 | String addPartition4 = "CALL kudu.system.add_range_partition('" + schema + "','" + tableName + "','" + ranges.range4 + "')"; 152 | queryRunner.execute(addPartition4); 153 | 154 | String dropPartition3 = addPartition3.replace(".add_range_partition(", ".drop_range_partition("); 155 | queryRunner.execute(dropPartition3); 156 | 157 | MaterializedResult result = queryRunner.execute("SHOW CREATE TABLE " + tableName); 158 | Assert.assertEquals(result.getRowCount(), 1); 159 | String createSQL = result.getMaterializedRows().get(0).getField(0).toString(); 160 | String rangesArray = "'[" + ranges.cmp1 + "," + ranges.cmp2 + "," + ranges.cmp4 + "]'"; 161 | rangesArray = rangesArray.replaceAll("\\s+", ""); 162 | String expectedRanges = "range_partitions = " + rangesArray; 163 | Assert.assertTrue(createSQL.contains(expectedRanges), createSQL + "\ncontains\n" + expectedRanges); 164 | String expectedNumReplicas = "num_replicas = 1,"; 165 | Assert.assertTrue(createSQL.contains(expectedNumReplicas), createSQL + "\ncontains\n" + expectedNumReplicas); 166 | } 167 | 168 | @BeforeClass 169 | public void setUp() { 170 | queryRunner = getQueryRunner(); 171 | } 172 | 173 | @AfterClass(alwaysRun = true) 174 | public final void destroy() { 175 | if (queryRunner != null) { 176 | queryRunner.close(); 177 | queryRunner = null; 178 | } 179 | } 180 | } 181 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/TypeHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu; 15 | 16 | import com.facebook.presto.spi.predicate.NullableValue; 17 | import com.facebook.presto.spi.type.BigintType; 18 | import com.facebook.presto.spi.type.BooleanType; 19 | import com.facebook.presto.spi.type.CharType; 20 | import com.facebook.presto.spi.type.DateType; 21 | import com.facebook.presto.spi.type.DecimalType; 22 | import com.facebook.presto.spi.type.Decimals; 23 | import com.facebook.presto.spi.type.DoubleType; 24 | import com.facebook.presto.spi.type.IntegerType; 25 | import com.facebook.presto.spi.type.RealType; 26 | import com.facebook.presto.spi.type.SmallintType; 27 | import com.facebook.presto.spi.type.TimestampType; 28 | import com.facebook.presto.spi.type.TinyintType; 29 | import com.facebook.presto.spi.type.Type; 30 | import com.facebook.presto.spi.type.VarbinaryType; 31 | import com.facebook.presto.spi.type.VarcharType; 32 | import io.airlift.slice.Slice; 33 | import io.airlift.slice.Slices; 34 | import org.apache.kudu.ColumnSchema; 35 | import org.apache.kudu.ColumnTypeAttributes; 36 | import org.apache.kudu.client.PartialRow; 37 | import org.apache.kudu.client.RowResult; 38 | 39 | import java.math.BigDecimal; 40 | 41 | import static io.airlift.slice.Slices.utf8Slice; 42 | import static io.airlift.slice.Slices.wrappedBuffer; 43 | import static java.lang.Float.floatToRawIntBits; 44 | import static java.lang.Float.intBitsToFloat; 45 | 46 | public class TypeHelper { 47 | 48 | public static org.apache.kudu.Type toKuduClientType(Type type) { 49 | if (type instanceof VarcharType) { 50 | return org.apache.kudu.Type.STRING; 51 | } else if (type == TimestampType.TIMESTAMP) { 52 | return org.apache.kudu.Type.UNIXTIME_MICROS; 53 | } else if (type == BigintType.BIGINT) { 54 | return org.apache.kudu.Type.INT64; 55 | } else if (type == IntegerType.INTEGER) { 56 | return org.apache.kudu.Type.INT32; 57 | } else if (type == SmallintType.SMALLINT) { 58 | return org.apache.kudu.Type.INT16; 59 | } else if (type == TinyintType.TINYINT) { 60 | return org.apache.kudu.Type.INT8; 61 | } else if (type == RealType.REAL) { 62 | return org.apache.kudu.Type.FLOAT; 63 | } else if (type == DoubleType.DOUBLE) { 64 | return org.apache.kudu.Type.DOUBLE; 65 | } else if (type == BooleanType.BOOLEAN) { 66 | return org.apache.kudu.Type.BOOL; 67 | } else if (type instanceof VarbinaryType) { 68 | return org.apache.kudu.Type.BINARY; 69 | } else if (type instanceof DecimalType) { 70 | return org.apache.kudu.Type.DECIMAL; 71 | } else if (type == DateType.DATE) { 72 | return org.apache.kudu.Type.STRING; 73 | } else if (type instanceof CharType) { 74 | return org.apache.kudu.Type.STRING; 75 | } else { 76 | throw new IllegalStateException("Type mapping implemented for Presto type: " + type); 77 | } 78 | } 79 | 80 | public static Type fromKuduColumn(ColumnSchema column) { 81 | return fromKuduClientType(column.getType(), column.getTypeAttributes()); 82 | } 83 | 84 | private static Type fromKuduClientType(org.apache.kudu.Type ktype, ColumnTypeAttributes attributes) { 85 | switch (ktype) { 86 | case STRING: 87 | return VarcharType.VARCHAR; 88 | case UNIXTIME_MICROS: 89 | return TimestampType.TIMESTAMP; 90 | case INT64: 91 | return BigintType.BIGINT; 92 | case INT32: 93 | return IntegerType.INTEGER; 94 | case INT16: 95 | return SmallintType.SMALLINT; 96 | case INT8: 97 | return TinyintType.TINYINT; 98 | case FLOAT: 99 | return RealType.REAL; 100 | case DOUBLE: 101 | return DoubleType.DOUBLE; 102 | case BOOL: 103 | return BooleanType.BOOLEAN; 104 | case BINARY: 105 | return VarbinaryType.VARBINARY; 106 | case DECIMAL: 107 | return DecimalType.createDecimalType(attributes.getPrecision(), attributes.getScale()); 108 | default: 109 | throw new IllegalStateException("Kudu type not implemented for " + ktype); 110 | } 111 | } 112 | 113 | public static Type mappedType(Type sourceType) { 114 | if (sourceType == DateType.DATE) { 115 | return VarcharType.VARCHAR; 116 | } else { 117 | return sourceType; 118 | } 119 | } 120 | 121 | public static NullableValue getColumnValue(Type type, PartialRow row, int i) { 122 | if (row.isNull(i)) { 123 | return NullableValue.asNull(type); 124 | } else { 125 | if (type instanceof VarcharType) { 126 | return NullableValue.of(type, utf8Slice(row.getString(i))); 127 | } else if (type == TimestampType.TIMESTAMP) { 128 | return NullableValue.of(type, row.getLong(i) / 1000); 129 | } else if (type == BigintType.BIGINT) { 130 | return NullableValue.of(type, row.getLong(i)); 131 | } else if (type == IntegerType.INTEGER) { 132 | return NullableValue.of(type, row.getInt(i)); 133 | } else if (type == SmallintType.SMALLINT) { 134 | return NullableValue.of(type, row.getShort(i)); 135 | } else if (type == TinyintType.TINYINT) { 136 | return NullableValue.of(type, row.getByte(i)); 137 | } else if (type == DoubleType.DOUBLE) { 138 | return NullableValue.of(type, row.getDouble(i)); 139 | } else if (type == RealType.REAL) { 140 | return NullableValue.of(type, (long) floatToRawIntBits(row.getFloat(i))); 141 | } else if (type == BooleanType.BOOLEAN) { 142 | return NullableValue.of(type, row.getBoolean(i)); 143 | } else if (type instanceof VarbinaryType) { 144 | return NullableValue.of(type, wrappedBuffer(row.getBinary(i))); 145 | } else if (type instanceof DecimalType) { 146 | return NullableValue.of(type, row.getDecimal(i)); 147 | } else { 148 | throw new IllegalStateException("Handling of type " + type + " is not implemented"); 149 | } 150 | } 151 | } 152 | 153 | public static Object getJavaValue(Type type, Object nativeValue) { 154 | if (type instanceof VarcharType) { 155 | return ((Slice) nativeValue).toStringUtf8(); 156 | } else if (type == TimestampType.TIMESTAMP) { 157 | return ((Long) nativeValue) * 1000; 158 | } else if (type == BigintType.BIGINT) { 159 | return nativeValue; 160 | } else if (type == IntegerType.INTEGER) { 161 | return ((Long) nativeValue).intValue(); 162 | } else if (type == SmallintType.SMALLINT) { 163 | return ((Long) nativeValue).shortValue(); 164 | } else if (type == TinyintType.TINYINT) { 165 | return ((Long) nativeValue).byteValue(); 166 | } else if (type == DoubleType.DOUBLE) { 167 | return nativeValue; 168 | } else if (type == RealType.REAL) { 169 | // conversion can result in precision lost 170 | return intBitsToFloat(((Long) nativeValue).intValue()); 171 | } else if (type == BooleanType.BOOLEAN) { 172 | return nativeValue; 173 | } else if (type instanceof VarbinaryType) { 174 | return ((Slice) nativeValue).toByteBuffer(); 175 | } else if (type instanceof DecimalType) { 176 | return nativeValue; 177 | } else { 178 | throw new IllegalStateException("Back conversion not implemented for " + type); 179 | } 180 | } 181 | 182 | public static Object getObject(Type type, RowResult row, int field) { 183 | if (row.isNull(field)) { 184 | return null; 185 | } else { 186 | if (type instanceof VarcharType) { 187 | return row.getString(field); 188 | } else if (type == TimestampType.TIMESTAMP) { 189 | return row.getLong(field) / 1000; 190 | } else if (type == BigintType.BIGINT) { 191 | return row.getLong(field); 192 | } else if (type == IntegerType.INTEGER) { 193 | return row.getInt(field); 194 | } else if (type == SmallintType.SMALLINT) { 195 | return row.getShort(field); 196 | } else if (type == TinyintType.TINYINT) { 197 | return row.getByte(field); 198 | } else if (type == DoubleType.DOUBLE) { 199 | return row.getDouble(field); 200 | } else if (type == RealType.REAL) { 201 | return row.getFloat(field); 202 | } else if (type == BooleanType.BOOLEAN) { 203 | return row.getBoolean(field); 204 | } else if (type instanceof VarbinaryType) { 205 | return Slices.wrappedBuffer(row.getBinary(field)); 206 | } else if (type instanceof DecimalType) { 207 | return row.getDecimal(field); 208 | } else { 209 | throw new IllegalStateException("getObject not implemented for " + type); 210 | } 211 | } 212 | } 213 | 214 | public static long getLong(Type type, RowResult row, int field) { 215 | if (type == TimestampType.TIMESTAMP) { 216 | return row.getLong(field) / 1000; 217 | } else if (type == BigintType.BIGINT) { 218 | return row.getLong(field); 219 | } else if (type == IntegerType.INTEGER) { 220 | return row.getInt(field); 221 | } else if (type == SmallintType.SMALLINT) { 222 | return row.getShort(field); 223 | } else if (type == TinyintType.TINYINT) { 224 | return row.getByte(field); 225 | } else if (type == RealType.REAL) { 226 | return floatToRawIntBits(row.getFloat(field)); 227 | } else if (type instanceof DecimalType) { 228 | DecimalType dtype = (DecimalType) type; 229 | if (dtype.isShort()) { 230 | return row.getDecimal(field).unscaledValue().longValue(); 231 | } else { 232 | throw new IllegalStateException("getLong not supported for long decimal: " + type); 233 | } 234 | } else { 235 | throw new IllegalStateException("getLong not implemented for " + type); 236 | } 237 | } 238 | 239 | public static boolean getBoolean(Type type, RowResult row, int field) { 240 | if (type == BooleanType.BOOLEAN) { 241 | return row.getBoolean(field); 242 | } else { 243 | throw new IllegalStateException("getBoolean not implemented for " + type); 244 | } 245 | } 246 | 247 | public static double getDouble(Type type, RowResult row, int field) { 248 | if (type == DoubleType.DOUBLE) { 249 | return row.getDouble(field); 250 | } else { 251 | throw new IllegalStateException("getDouble not implemented for " + type); 252 | } 253 | } 254 | 255 | public static Slice getSlice(Type type, RowResult row, int field) { 256 | if (type instanceof VarcharType) { 257 | return Slices.utf8Slice(row.getString(field)); 258 | } else if (type instanceof VarbinaryType) { 259 | return Slices.wrappedBuffer(row.getBinary(field)); 260 | } else if (type instanceof DecimalType) { 261 | BigDecimal dec = row.getDecimal(field); 262 | return Decimals.encodeScaledValue(dec); 263 | } else { 264 | throw new IllegalStateException("getSlice not implemented for " + type); 265 | } 266 | } 267 | } -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Presto-Kudu 2 | The [Presto](https://prestosql.io/) Kudu connector allows querying, inserting and deleting data in [Apache Kudu](https://kudu.apache.org/) 3 | 4 | ## Integration into PrestoDB distribution 5 | 6 | Starting with Presto 0.209 the presto-kudu connector is integrated into the Presto distribution. 7 | Syntax for creating tables has changed, but the functionality is the same. 8 | Please see [Presto Documentation / Kudu Connector](https://prestosql.io/docs/current/connector/kudu.html) for more details. 9 | 10 | ### Migration 11 | 12 | If you want to migrate from Presto <= 0.208 with this Kudu connector to a newer Presto version with the integrated Kudu connector, 13 | perform following steps: 14 | 15 | - Stop Presto 0.208 16 | 17 | - Delete the table `$schemas` from Kudu using the Java or Python Kudu client. 18 | This table will be recreated automatically. 19 | 20 | - If you want to continue using the schema emulation used in this old connector, 21 | set following property in the `kudu.properties`: 22 | 23 | ```conf 24 | kudu.schema-emulation.prefix= 25 | ``` 26 | 27 | - Start Presto >= 0.209 28 | 29 | ## Compatibility 30 | 31 | | Version | Compatibility | Details | 32 | | ------- | --------------| ------------- | 33 | | Apache Kudu 1.8.0 | yes | tests ok | 34 | | Apache Kudu 1.7.0/1.7.1 | yes | by full API- and ABI-compatibility of Kudu Java Client 1.8.0 | 35 | | Apache Kudu 1.6.0 | yes | by full API- and ABI-compatibility of Kudu Java Client 1.8.0 | 36 | | Apache Kudu 1.5.0 | yes | by full API- and ABI-compatibility of Kudu Java Client 1.8.0 | 37 | | Apache Kudu 1.4.0 | yes | by full API- and ABI-compatibility of Kudu Java Client 1.8.0 | 38 | | | | | | 39 | | Presto 0.208 | yes | tests ok | 40 | 41 | Support for older Presto versions see [release history](https://github.com/MartinWeindel/presto-kudu/releases) 42 | 43 | ## Installation 44 | 45 | Please follow the below steps to query Apache Kudu in Presto. 46 | 47 | ### Deploying Kudu server 48 | Follow installation guide at [Apache Kudu](https://kudu.apache.org/). 49 | 50 | If you want to deploy Kudu 1.8.0 on RHE 7 or CentOS 7, you may also be 51 | interessed in my binary build project [kudu-rpm](https://github.com/MartinWeindel/kudu-rpm/releases/tag/v1.8.0-1). 52 | 53 | ### Deploying Presto server 54 | Install Presto according to the documentation: https://prestosql.io/docs/current/installation/deployment.html 55 | 56 | ### Download Presto-Kudu connector 57 | [Download current release](https://github.com/MartinWeindel/presto-kudu/wiki/Download) 58 | 59 | ### Configuring Apache Kudu connector 60 | * Go to the directory `$PRESTO_HOME$/plugin` 61 | * Extract the content of `presto-kudu-XXX.zip` to this folder 62 | * Rename the extracted folder `presto-kudu-XXX` to `kudu` 63 | * Create a file name `kudu.properties` in `$PRESTO_HOME/etc/catalog/`: 64 | ``` 65 | connector.name=kudu 66 | 67 | ## List of Kudu master addresses, at least one is needed (comma separated) 68 | ## Supported formats: example.com, example.com:7051, 192.0.2.1, 192.0.2.1:7051, 69 | ## [2001:db8::1], [2001:db8::1]:7051, 2001:db8::1 70 | kudu.client.master-addresses=localhost 71 | 72 | ## Optional restriction of tablets for specific tenant. 73 | ## If a tenant is set, only Kudu tablets starting with `.` will 74 | ## be visible in Presto 75 | #kudu.session.tenant=mytenant 76 | 77 | ####################### 78 | ### Advanced Kudu Java client configuration 79 | ####################### 80 | 81 | ## Default timeout used for administrative operations (e.g. createTable, deleteTable, etc.) 82 | #kudu.client.defaultAdminOperationTimeout = 30s 83 | 84 | ## Default timeout used for user operations 85 | #kudu.client.defaultOperationTimeout = 30s 86 | 87 | ## Default timeout to use when waiting on data from a socket 88 | #kudu.client.defaultSocketReadTimeout = 10s 89 | 90 | ## Disable Kudu client's collection of statistics. 91 | #kudu.client.disableStatistics = false 92 | ``` 93 | 94 | 95 | ### Query kudu in CLI of presto 96 | * Download presto cli client following: https://prestosql.io/docs/current/installation/cli.html 97 | 98 | * Start CLI: 99 | 100 | ```bash 101 | ./presto-cli --server localhost:8086 --catalog kudu --schema default 102 | ``` 103 | Replace the hostname, port and schema name with your own. 104 | 105 | ## Querying Data 106 | A Kudu table named `mytable` is available in Presto as table `kudu.default.mytable`. 107 | A Kudu table containing a dot is considered as a schema/table combination, e.g. 108 | `dev.mytable` is mapped to the Presto table `kudu.dev.mytable. 109 | Only Kudu table names in lower case are currently supported. 110 | 111 | 112 | - Now you can use any Kudu table, if it is lower case and contains no dots. 113 | - Alternatively you can create a users table with 114 | ```sql 115 | CREATE TABLE users ( 116 | user_id int, 117 | first_name varchar, 118 | last_name varchar 119 | ) WITH ( 120 | column_design = '{"user_id": {"key": true}}', 121 | partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', 122 | num_replicas = 1 123 | ); 124 | ``` 125 | On creating a Kudu table you must/can specify addition information about 126 | the primary key, encoding, and compression of columns and hash or range partitioning, 127 | and the number of replicas. Details see below in section "Create Kudu Table". 128 | 129 | - The table can be described using 130 | ```sql 131 | DESCRIBE kudu.default.users; 132 | ``` 133 | You should get something like 134 | ``` 135 | Column | Type | Extra | Comment 136 | ------------+---------+-------------------------------------------------------------------+--------- 137 | user_id | integer | key, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | 138 | first_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | 139 | last_name | varchar | nullable, encoding=AUTO_ENCODING, compression=DEFAULT_COMPRESSION | 140 | (3 rows) 141 | ``` 142 | 143 | - Insert some data with 144 | ```sql 145 | INSERT INTO users VALUES (1, 'Donald', 'Duck'), (2, 'Mickey', 'Mouse'); 146 | ``` 147 | 148 | - Select the inserted data 149 | ```sql 150 | SELECT * FROM users; 151 | ``` 152 | 153 | 154 | ## Data Type Mapping 155 | The data types of Presto and Kudu are mapped as far as possible: 156 | 157 | | Presto Data Type | Kudu Data Type | Comment | 158 | | ---------------- | -------------- | ------- | 159 | | `BOOLEAN` | `BOOL` | | 160 | | `TINYINT` | `INT8` | | 161 | | `SMALLINT` | `INT16` | | 162 | | `INTEGER` | `INT32` | | 163 | | `BIGINT` | `INT64` | | 164 | | `REAL` | `FLOAT` | | 165 | | `DOUBLE` | `DOUBLE` | | 166 | | `VARCHAR` | `STRING` | see note 1 | 167 | | `VARBINARY` | `BINARY` | see note 1 | 168 | | `TIMESTAMP` | `UNIXTIME_MICROS` | µs resolution in Kudu column is reduced to ms resolution | 169 | | `DECIMAL` | `DECIMAL` | only supported for Kudu server >= 1.7.0 | 170 | | `CHAR` | - | not supported, see note 2 | 171 | | `DATE` | - | not supported, see note 2 | 172 | | `TIME` | - | not supported | 173 | | `JSON` | - | not supported | 174 | | `TIME WITH TIMEZONE` | - | not supported | 175 | | `TIMESTAMP WITH TIMEZONE` | - | not supported | 176 | | `INTERVAL YEAR TO MONTH` | - | not supported | 177 | | `INTERVAL DAY TO SECOND` | - | not supported | 178 | | `ARRAY` | - | not supported | 179 | | `MAP` | - | not supported | 180 | | `IPADDRESS` | - | not supported | 181 | 182 | #### Note 1 183 | On performing `CREATE TABLE ... AS ...` from a Presto table to Kudu, 184 | the optional maximum length is lost 185 | 186 | #### Note 2 187 | On performing `CREATE TABLE ... AS ...` from a Presto table to Kudu, 188 | a `DATE` or `CHAR` column is converted to `STRING` 189 | 190 | ## Supported Presto SQL statements 191 | | Presto SQL statement | Supported | Comment | 192 | | -------------------- | --------- | ------- | 193 | | `SELECT` | [x] | | 194 | | `INSERT INTO ... VALUES` | [x] | behaves like `upsert` | 195 | | `INSERT INTO ... SELECT ... ` | [x] | behaves like `upsert` | 196 | | `DELETE` | [x] | | 197 | | `CREATE SCHEMA` | [x] | | 198 | | `DROP SCHEMA` | [x] | | 199 | | `CREATE TABLE` | [x] | | 200 | | `CREATE TABLE ... AS` | [x] | | 201 | | `DROP TABLE` | [x] | | 202 | | `ALTER TABLE ... RENAME TO ...` | [x] | | 203 | | `ALTER TABLE ... RENAME COLUMN ...` | [x] | if not part of primary key | 204 | | `ALTER TABLE ... ADD COLUMN ...` | [x] | | 205 | | `ALTER TABLE ... DROP COLUMN ...` | [x] | if not part of primary key | 206 | | `SHOW SCHEMAS` | [x] | | 207 | | `SHOW TABLES` | [x] | | 208 | | `SHOW CREATE TABLE` | [x] | | 209 | | `SHOW COLUMNS FROM` | [x] | | 210 | | `DESCRIBE` | [x] | same as `SHOW COLUMNS FROM`| 211 | | `CALL kudu.system.add_range_partition` | [x] | add range partition to an existing table | 212 | | `CALL kudu.system.drop_range_partition` | [x] | drop an existing range partition from a table | 213 | 214 | Currently not supported are `SHOW PARTITIONS FROM ...`, `ALTER SCHEMA ... RENAME` 215 | 216 | ## Create Kudu Table with `CREATE TABLE` 217 | On creating a Kudu Table you need to provide following table properties: 218 | - `column_design` 219 | - `partition_design` (optional) 220 | - `num_replicas` (optional, defaults to 3) 221 | 222 | Example: 223 | ```sql 224 | CREATE TABLE users ( 225 | user_id int, 226 | first_name varchar, 227 | last_name varchar 228 | ) WITH ( 229 | column_design = '{"user_id": {"key": true}}', 230 | partition_design = '{"hash":[{"columns":["user_id"], "buckets": 2}]}', 231 | num_replicas = 1 232 | ); 233 | ``` 234 | 235 | ### Table property `column_design` 236 | With the column design table property you define the columns for the primary key. 237 | Additionally you can overwrite the encoding and compression of every single column. 238 | 239 | The value of this property must be a string of a valid JSON object. 240 | The keys are the columns and the values is a JSON object with the columns properties 241 | to set, i.e. 242 | 243 | ``` 244 | '{"": {"": , ...}, ...}'` 245 | ``` 246 | 247 | | Column property name | Value | Comment | 248 | | -------------------- | ----- | ------- | 249 | | `key` | `true` or `false` | if column belongs to primary key, default: `false` | 250 | | `nullable` | `true` or `false` | if column is nullable, default: `true` for non-key columns, key columns must not be nullable | 251 | | `encoding` | "string value" | See Apache Kudu documentation: [Column encoding](https://kudu.apache.org/docs/schema_design.html#encoding) | 252 | | `compression` | "string value" | See Apache Kudu documentation: [Column compression](https://kudu.apache.org/docs/schema_design.html#compression) | 253 | 254 | Example: 255 | ``` 256 | '{"column1": {"key": true, "encoding": "dictionary", "compression": "LZ4"}, "column2": {...}}' 257 | ``` 258 | 259 | ### Table property `partition_design` 260 | With the partition design table property you define the partition layout. 261 | In Apache Kudu you can define multiple hash partitions and at most one range partition. 262 | Details see Apache Kudu documentation: [Partitioning](https://kudu.apache.org/docs/schema_design.html#partitioning) 263 | 264 | The value of this property must be a string of a valid JSON object. 265 | The keys are either `hash` or `range` or both, i.e. 266 | 267 | ``` 268 | '{"hash": [{...},...], "range": {...}}'` 269 | ``` 270 | #### Hash partitioning 271 | You can provide multiple hash partition groups in Apache Kudu. 272 | Each group consists of a list of column names and the number of buckets. 273 | 274 | Example: 275 | ``` 276 | '{"hash": [{"columns": ["region", "name"], "buckets": 5}]}' 277 | ``` 278 | This defines a hash partition with the columns "region" and "name", 279 | distributed over 5 buckets. All partition columns must be part of 280 | the primary key. 281 | 282 | #### Range partitioning 283 | You can provide at most one range partition in Apache Kudu. 284 | It consists of a list of columns. The ranges themselves are given either 285 | in the table property `range_partitions`. Alternatively, the 286 | procedures `kudu.system.add_range_partition` and `kudu.system.drop_range_partition` 287 | can be used to manage range partitions for existing tables. 288 | For both ways see below for more details. 289 | 290 | Example: 291 | ``` 292 | '{"range": {"columns": ["event_time"]}}' 293 | ``` 294 | 295 | Defines range partitioning on the column "event". 296 | 297 | To add concrete range partitions use either the table property `range_partitions` 298 | or call the procedure . 299 | 300 | ### Table property `range_partitions` 301 | With the `range_partitions` table property you specify the concrete range partitions to be 302 | created. The range partition definition itself must be given in the table 303 | property `partition_design` separately. 304 | 305 | 306 | 307 | Example: 308 | ```sql 309 | CREATE TABLE events ( 310 | serialno varchar, 311 | event_time timestamp, 312 | message varchar 313 | ) WITH ( 314 | column_design = '{"serialno": {"key": true}, "event_time": {"key": true}}', 315 | partition_design = '{"hash":[{"columns":["serialno"], "buckets": 4}], 316 | "range": {"columns":["event_time"]}}', 317 | range_partitions = '[{"lower": null, "upper": "2017-01-01T00:00:00"}, 318 | {"lower": "2017-01-01T00:00:00", "upper": "2017-07-01T00:00:00"}, 319 | {"lower": "2017-07-01T00:00:00", "upper": "2018-01-01T00:00:00"}]', 320 | num_replicas = 1 321 | ); 322 | ``` 323 | This creates a table with a hash partition on column `serialno` with 4 buckets and range partitioning 324 | on column `event_time`. Additionally three range partitions are created: 325 | 1. for all event_times before the year 2017 (lower bound = `null` means it is unbound) 326 | 2. for the first half of the year 2017 327 | 3. for the second half the year 2017 328 | This means any try to add rows with `event_time` of year 2018 or greater will fail, 329 | as no partition is defined. 330 | 331 | #### Managing range partitions 332 | For existing tables, there are procedures to add and drop a range partition. 333 | 334 | - adding a range partition 335 | ```sql 336 | CALL kudu.system.add_range_partition(, , ), 337 | ``` 338 | 339 | - dropping a range partition 340 | ```sql 341 | CALL kudu.system.drop_range_partition(,
, ) 342 | ``` 343 | 344 | - ``: schema of the table 345 | - `
`: table names 346 | - ``: lower and upper bound of the range partition 347 | as json string in the form `'{"lower": , "upper": }'`, or if the range 348 | partition has multiple columns: `'{"lower": [,...], "upper": [,...]}'`. 349 | The concrete literal for lower and upper bound values are depending on the 350 | column types. 351 | 352 | Examples: 353 | 354 | | Presto Data Type | JSON string example | 355 | | ---------------- | ------------------- | 356 | | BIGINT | '{"lower": 0, "upper": 1000000}' | 357 | | SMALLINT | '{"lower": 10, "upper": null}' | 358 | | VARCHAR | '{"lower": "A", "upper": "M"}' | 359 | | TIMESTAMP | '{"lower": "2018-02-01T00:00:00.000", "upper": "2018-02-01T12:00:00.000"}' | 360 | | BOOLEAN | '{"lower": false, "upper": true}' | 361 | | VARBINARY | values encoded as base64 strings | 362 | 363 | To specified an unbounded bound, use the value `null`. 364 | 365 | 366 | Example: 367 | ```sql 368 | CALL kudu.system.add_range_partition('myschema', 'events', '{"lower": "2018-01-01", "upper": "2018-06-01"}') 369 | ``` 370 | This would add a range partition for a table `events` in the schema `myschema` with 371 | the lower bound `2018-01-01` (more exactly `2018-01-01T00:00:00.000`) and the upper bound `2018-07-01`. 372 | 373 | Use the sql statement `SHOW CREATE TABLE` to request the existing range partitions (they are shown 374 | in the table property `range_partitions`). 375 | 376 | ## Known limitations 377 | - Only lower case table and column names in Kudu are supported 378 | - As schemas are not directly supported by Kudu, a special table named `$schemas` 379 | is created in Kudu when using this connector 380 | - Using a secured Kudu cluster has not been tested. 381 | 382 | ## Build 383 | The Presto-Kudu connector is a standard Maven project. 384 | Under Linux, simply run the following command from the project root directory: 385 | 386 | ```bash 387 | mvn -DskipTests clean package 388 | ``` 389 | 390 | Building the package under Windows is currently not supported, as the maven plugin 391 | `maven-presto-plugin` has an open issue with Windows. 392 | 393 | To run the build with tests, it is assumed that Kudu master server 394 | (and at least one Kudu tablet server) runs on localhost. 395 | If you have Docker installed on your machine, you can use following steps: 396 | ```bash 397 | docker run --rm -d --name apache-kudu --net=host usuresearch/kudu-docker-slim:release-v1.8.0-1 398 | mvn clean package 399 | docker stop apache-kudu 400 | ``` 401 | -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/KuduMetadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package ml.littlebulb.presto.kudu; 19 | 20 | import com.facebook.presto.spi.ColumnHandle; 21 | import com.facebook.presto.spi.ColumnMetadata; 22 | import com.facebook.presto.spi.ConnectorInsertTableHandle; 23 | import com.facebook.presto.spi.ConnectorNewTableLayout; 24 | import com.facebook.presto.spi.ConnectorOutputTableHandle; 25 | import com.facebook.presto.spi.ConnectorSession; 26 | import com.facebook.presto.spi.ConnectorTableHandle; 27 | import com.facebook.presto.spi.ConnectorTableLayout; 28 | import com.facebook.presto.spi.ConnectorTableLayoutHandle; 29 | import com.facebook.presto.spi.ConnectorTableLayoutResult; 30 | import com.facebook.presto.spi.ConnectorTableMetadata; 31 | import com.facebook.presto.spi.Constraint; 32 | import com.facebook.presto.spi.SchemaTableName; 33 | import com.facebook.presto.spi.SchemaTablePrefix; 34 | import com.facebook.presto.spi.connector.ConnectorMetadata; 35 | import com.facebook.presto.spi.connector.ConnectorOutputMetadata; 36 | import com.facebook.presto.spi.statistics.ComputedStatistics; 37 | import com.facebook.presto.spi.type.Type; 38 | import com.facebook.presto.spi.type.VarbinaryType; 39 | import com.facebook.presto.spi.type.VarcharType; 40 | import com.google.common.collect.ImmutableList; 41 | import com.google.common.collect.ImmutableMap; 42 | import io.airlift.slice.Slice; 43 | import ml.littlebulb.presto.kudu.properties.KuduTableProperties; 44 | import org.apache.kudu.ColumnSchema; 45 | import org.apache.kudu.Schema; 46 | import org.apache.kudu.client.KuduTable; 47 | 48 | import java.util.ArrayList; 49 | import java.util.Collection; 50 | import java.util.HashMap; 51 | import java.util.List; 52 | import java.util.Map; 53 | import java.util.Optional; 54 | import java.util.Set; 55 | 56 | import javax.inject.Inject; 57 | 58 | import static com.google.common.collect.ImmutableList.toImmutableList; 59 | import static java.util.Objects.requireNonNull; 60 | import static ml.littlebulb.presto.kudu.Types.checkType; 61 | 62 | public class KuduMetadata implements ConnectorMetadata { 63 | private final String connectorId; 64 | private KuduClientSession clientSession; 65 | 66 | @Inject 67 | public KuduMetadata(KuduConnectorId connectorId, KuduClientSession clientSession) { 68 | this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); 69 | this.clientSession = requireNonNull(clientSession, "clientSession is null"); 70 | } 71 | 72 | @Override 73 | public List listSchemaNames(ConnectorSession session) { 74 | return clientSession.listSchemaNames(); 75 | } 76 | 77 | @Override 78 | public List listTables(ConnectorSession session, String schemaNameOrNull) { 79 | return clientSession.listTables(schemaNameOrNull); 80 | } 81 | 82 | @Override 83 | public Map> listTableColumns(ConnectorSession session, 84 | SchemaTablePrefix prefix) { 85 | requireNonNull(prefix, "SchemaTablePrefix is null"); 86 | 87 | List tables; 88 | if (prefix.getSchemaName() == null) { 89 | tables = listTables(session, Optional.empty()); 90 | } else if (prefix.getTableName() == null) { 91 | tables = listTables(session, prefix.getSchemaName()); 92 | } else { 93 | tables = ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); 94 | } 95 | 96 | ImmutableMap.Builder> columns = ImmutableMap.builder(); 97 | for (SchemaTableName tableName : tables) { 98 | KuduTableHandle tableHandle = getTableHandle(session, tableName); 99 | ConnectorTableMetadata tableMetadata = getTableMetadata(tableHandle); 100 | columns.put(tableName, tableMetadata.getColumns()); 101 | } 102 | return columns.build(); 103 | } 104 | 105 | 106 | private ConnectorTableMetadata getTableMetadata(KuduTableHandle tableHandle) { 107 | KuduTable table = tableHandle.getTable(clientSession); 108 | Schema schema = table.getSchema(); 109 | 110 | List columnsMetaList = schema.getColumns().stream() 111 | .filter(col -> !col.isKey() || !col.getName().equals(KuduColumnHandle.ROW_ID)) 112 | .map(col -> { 113 | StringBuilder extra = new StringBuilder(); 114 | if (col.isKey()) { 115 | extra.append("key, "); 116 | } else if (col.isNullable()) { 117 | extra.append("nullable, "); 118 | } 119 | if (col.getEncoding() != null) { 120 | extra.append("encoding=").append(col.getEncoding().name()).append(", "); 121 | } 122 | if (col.getCompressionAlgorithm() != null) { 123 | extra.append("compression=").append(col.getCompressionAlgorithm().name()).append(", "); 124 | } 125 | if (extra.length() > 2) { 126 | extra.setLength(extra.length() - 2); 127 | } 128 | Type prestoType = TypeHelper.fromKuduColumn(col); 129 | return new ColumnMetadata(col.getName(), prestoType, null, extra.toString(), false); 130 | }).collect(toImmutableList()); 131 | 132 | Map properties = clientSession.getTableProperties(tableHandle); 133 | return new ConnectorTableMetadata(tableHandle.getSchemaTableName(), columnsMetaList, properties); 134 | } 135 | 136 | private KuduTableHandle fromConnectorTableHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { 137 | return checkType(tableHandle, KuduTableHandle.class, "tableHandle"); 138 | } 139 | 140 | @Override 141 | public Map getColumnHandles(ConnectorSession session, 142 | ConnectorTableHandle connectorTableHandle) { 143 | KuduTableHandle tableHandle = fromConnectorTableHandle(session, connectorTableHandle); 144 | Schema schema = clientSession.getTableSchema(tableHandle); 145 | 146 | ImmutableMap.Builder columnHandles = ImmutableMap.builder(); 147 | for (int i = 0; i < schema.getColumnCount(); i++) { 148 | ColumnSchema col = schema.getColumnByIndex(i); 149 | String name = col.getName(); 150 | Type type = TypeHelper.fromKuduColumn(col); 151 | KuduColumnHandle columnHandle = new KuduColumnHandle(name, i, type); 152 | columnHandles.put(name, columnHandle); 153 | } 154 | 155 | return columnHandles.build(); 156 | } 157 | 158 | @Override 159 | public ColumnMetadata getColumnMetadata(ConnectorSession session, 160 | ConnectorTableHandle tableHandle, ColumnHandle columnHandle) { 161 | fromConnectorTableHandle(session, tableHandle); 162 | KuduColumnHandle kuduColumnHandle = checkType(columnHandle, KuduColumnHandle.class, "columnHandle"); 163 | if (kuduColumnHandle.isVirtualRowId()) { 164 | return new ColumnMetadata(KuduColumnHandle.ROW_ID, VarbinaryType.VARBINARY, null, true); 165 | } else { 166 | return kuduColumnHandle.getColumnMetadata(); 167 | } 168 | } 169 | 170 | @Override 171 | public KuduTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) { 172 | try { 173 | KuduTable table = clientSession.openTable(schemaTableName); 174 | return new KuduTableHandle(connectorId, schemaTableName, table); 175 | } catch (Exception e) { 176 | return null; 177 | } 178 | } 179 | 180 | @Override 181 | public List getTableLayouts(ConnectorSession session, 182 | ConnectorTableHandle tableHandle, 183 | Constraint constraint, 184 | Optional> desiredColumns) { 185 | KuduTableHandle handle = fromConnectorTableHandle(session, tableHandle); 186 | ConnectorTableLayout layout = new ConnectorTableLayout( 187 | new KuduTableLayoutHandle(handle, constraint.getSummary(), desiredColumns)); 188 | return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); 189 | } 190 | 191 | @Override 192 | public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) { 193 | return new ConnectorTableLayout(handle); 194 | } 195 | 196 | @Override 197 | public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) { 198 | return getTableMetadataInternal(session, tableHandle); 199 | } 200 | 201 | private ConnectorTableMetadata getTableMetadataInternal(ConnectorSession session, 202 | ConnectorTableHandle tableHandle) { 203 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 204 | return getTableMetadata(kuduTableHandle); 205 | } 206 | 207 | @Override 208 | public void createSchema(ConnectorSession session, String schemaName, Map properties) { 209 | clientSession.createSchema(schemaName); 210 | } 211 | 212 | @Override 213 | public void dropSchema(ConnectorSession session, String schemaName) { 214 | clientSession.dropSchema(schemaName); 215 | } 216 | 217 | @Override 218 | public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting) { 219 | clientSession.createTable(tableMetadata, ignoreExisting); 220 | } 221 | 222 | @Override 223 | public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle) { 224 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 225 | clientSession.dropTable(kuduTableHandle.getSchemaTableName()); 226 | } 227 | 228 | @Override 229 | public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTableName) { 230 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 231 | clientSession.renameTable(kuduTableHandle.getSchemaTableName(), newTableName); 232 | } 233 | 234 | @Override 235 | public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnMetadata column) { 236 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 237 | clientSession.addColumn(kuduTableHandle.getSchemaTableName(), column); 238 | } 239 | 240 | @Override 241 | public void dropColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle column) { 242 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 243 | KuduColumnHandle kuduColumnHandle = (KuduColumnHandle) column; 244 | clientSession.dropColumn(kuduTableHandle.getSchemaTableName(), kuduColumnHandle.getName()); 245 | } 246 | 247 | @Override 248 | public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle source, 249 | String target) { 250 | KuduTableHandle kuduTableHandle = fromConnectorTableHandle(session, tableHandle); 251 | KuduColumnHandle kuduColumnHandle = (KuduColumnHandle) source; 252 | clientSession.renameColumn(kuduTableHandle.getSchemaTableName(), kuduColumnHandle.getName(), target); 253 | } 254 | 255 | @Override 256 | public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle connectorTableHandle) { 257 | KuduTableHandle tableHandle = fromConnectorTableHandle(session, connectorTableHandle); 258 | 259 | KuduTable table = tableHandle.getTable(clientSession); 260 | Schema schema = table.getSchema(); 261 | 262 | List columns = schema.getColumns(); 263 | List columnNames = columns.stream().map(ColumnSchema::getName).collect(toImmutableList()); 264 | List columnTypes = columns.stream() 265 | .map(TypeHelper::fromKuduColumn).collect(toImmutableList()); 266 | 267 | return new KuduInsertTableHandle( 268 | connectorId, 269 | tableHandle.getSchemaTableName(), 270 | columnNames, 271 | columnTypes, 272 | table); 273 | } 274 | 275 | @Override 276 | public Optional finishInsert(ConnectorSession session, 277 | ConnectorInsertTableHandle insertHandle, 278 | Collection fragments, 279 | Collection computedStatistics) { 280 | return Optional.empty(); 281 | } 282 | 283 | @Override 284 | public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, 285 | Optional layout) { 286 | boolean generateUUID = !tableMetadata.getProperties().containsKey(KuduTableProperties.PARTITION_DESIGN); 287 | ConnectorTableMetadata finalTableMetadata = tableMetadata; 288 | if (generateUUID) { 289 | String rowId = KuduColumnHandle.ROW_ID; 290 | List copy = new ArrayList<>(tableMetadata.getColumns()); 291 | copy.add(0, new ColumnMetadata(rowId, VarcharType.VARCHAR, "key=true", null, true)); 292 | List finalColumns = ImmutableList.copyOf(copy); 293 | Map propsCopy = new HashMap<>(tableMetadata.getProperties()); 294 | propsCopy.put(KuduTableProperties.COLUMN_DESIGN, "{\"" + rowId + "\": {\"key\": true}}"); 295 | propsCopy.put(KuduTableProperties.PARTITION_DESIGN, "{\"hash\": [{\"columns\": [\"" + rowId + "\"], " + 296 | "\"buckets\": 2}]}"); 297 | propsCopy.put(KuduTableProperties.NUM_REPLICAS, 1); 298 | Map finalProperties = ImmutableMap.copyOf(propsCopy); 299 | finalTableMetadata = new ConnectorTableMetadata(tableMetadata.getTable(), 300 | finalColumns, finalProperties, tableMetadata.getComment()); 301 | } 302 | KuduTable table = clientSession.createTable(finalTableMetadata, false); 303 | 304 | Schema schema = table.getSchema(); 305 | 306 | List columns = schema.getColumns(); 307 | List columnNames = columns.stream().map(ColumnSchema::getName).collect(toImmutableList()); 308 | List columnTypes = columns.stream() 309 | .map(TypeHelper::fromKuduColumn).collect(toImmutableList()); 310 | List columnOriginalTypes = finalTableMetadata.getColumns().stream() 311 | .map(ColumnMetadata::getType).collect(toImmutableList()); 312 | 313 | return new KuduOutputTableHandle( 314 | connectorId, 315 | finalTableMetadata.getTable(), 316 | columnOriginalTypes, 317 | columnNames, 318 | columnTypes, 319 | generateUUID, 320 | table); 321 | } 322 | 323 | @Override 324 | public Optional finishCreateTable(ConnectorSession session, 325 | ConnectorOutputTableHandle tableHandle, 326 | Collection fragments, 327 | Collection computedStatistics) { 328 | return Optional.empty(); 329 | } 330 | 331 | @Override 332 | public ColumnHandle getUpdateRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { 333 | return KuduColumnHandle.ROW_ID_HANDLE; 334 | } 335 | 336 | @Override 337 | public ConnectorTableHandle beginDelete(ConnectorSession session, ConnectorTableHandle tableHandle) { 338 | return fromConnectorTableHandle(session, tableHandle); 339 | } 340 | 341 | @Override 342 | public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHandle, Collection fragments) { 343 | } 344 | 345 | @Override 346 | public boolean supportsMetadataDelete(ConnectorSession session, ConnectorTableHandle tableHandle, 347 | ConnectorTableLayoutHandle tableLayoutHandle) { 348 | return false; 349 | } 350 | 351 | 352 | } -------------------------------------------------------------------------------- /src/main/java/ml/littlebulb/presto/kudu/properties/KuduTableProperties.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package ml.littlebulb.presto.kudu.properties; 15 | 16 | import com.facebook.presto.spi.session.PropertyMetadata; 17 | import com.fasterxml.jackson.core.type.TypeReference; 18 | import com.fasterxml.jackson.databind.ObjectMapper; 19 | import com.google.common.collect.ImmutableList; 20 | import org.apache.kudu.ColumnSchema; 21 | import org.apache.kudu.Schema; 22 | import org.apache.kudu.Type; 23 | import org.apache.kudu.client.KeyEncoderAccessor; 24 | import org.apache.kudu.client.KuduTable; 25 | import org.apache.kudu.client.LocatedTablet; 26 | import org.apache.kudu.client.PartialRow; 27 | import org.apache.kudu.client.Partition; 28 | import org.apache.kudu.client.PartitionSchema; 29 | import org.apache.kudu.shaded.com.google.common.base.Predicates; 30 | import org.apache.kudu.shaded.com.google.common.collect.Iterators; 31 | import org.joda.time.DateTimeZone; 32 | import org.joda.time.format.ISODateTimeFormat; 33 | 34 | import java.io.IOException; 35 | import java.math.BigDecimal; 36 | import java.util.ArrayList; 37 | import java.util.Base64; 38 | import java.util.HashMap; 39 | import java.util.Iterator; 40 | import java.util.LinkedHashMap; 41 | import java.util.List; 42 | import java.util.Map; 43 | import java.util.Optional; 44 | 45 | import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; 46 | import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; 47 | import static com.google.common.collect.ImmutableList.toImmutableList; 48 | import static java.util.Objects.requireNonNull; 49 | 50 | /** 51 | * Class contains all table properties for the Kudu connector. Used when creating a table: 52 | *
 53 |  * CREATE TABLE foo (a VARCHAR, b INT)
 54 |  * WITH (
 55 |  * column_design = '{"a": {"key": true}, "b": {"encoding": "bitshuffle"}}',
 56 |  * partition_design = '{"hash":[{"columns":["a"], "buckets": 2}]}',
 57 |  * num_replicas = 1
 58 |  * );
 59 |  * 
60 | *
or with range partitioning
61 | *
 62 |  * CREATE TABLE foo (a VARCHAR, b INT)
 63 |  * WITH (
 64 |  * column_design = '{"a": {"key": true}}',
 65 |  * partition_design = '{"range":{"columns":["a"]}}',
 66 |  * range_partitions = '[{"lower": null, "upper": "Am"}, {"lower": "Am", "upper": "Bs"}]',
 67 |  * num_replicas = 1
 68 |  * );
 69 |  * 
70 | */ 71 | public final class KuduTableProperties { 72 | public static final String COLUMN_DESIGN = "column_design"; 73 | public static final String PARTITION_DESIGN = "partition_design"; 74 | public static final String RANGE_PARTITIONS = "range_partitions"; 75 | public static final String NUM_REPLICAS = "num_replicas"; 76 | 77 | private static final ObjectMapper mapper = new ObjectMapper(); 78 | 79 | private static final long DEFAULT_DEADLINE = 20000; // deadline for retrieving range partitions in milliseconds 80 | 81 | private final List> tableProperties; 82 | 83 | 84 | public KuduTableProperties() { 85 | PropertyMetadata s1 = stringProperty( 86 | COLUMN_DESIGN, 87 | "Kudu-specific column design (key, encoding, and compression) as JSON, like {\"column1\": {\"key\": true, \"encoding\": \"dictionary\", \"compression\": \"LZ4\"}, \"column2\": {...}}", 88 | null, 89 | false); 90 | 91 | PropertyMetadata s2 = stringProperty( 92 | PARTITION_DESIGN, 93 | "Partition design (hash partition(s) and/or range partition) as JSON.", 94 | null, 95 | false); 96 | 97 | PropertyMetadata s3 = integerProperty( 98 | NUM_REPLICAS, 99 | "Number of tablet replicas. Default 3.", 100 | 3, 101 | false); 102 | 103 | PropertyMetadata s4 = stringProperty( 104 | RANGE_PARTITIONS, 105 | "Initial range partitions as JSON", 106 | null, 107 | false); 108 | 109 | tableProperties = ImmutableList.of(s1, s2, s3, s4); 110 | } 111 | 112 | public List> getTableProperties() { 113 | return tableProperties; 114 | } 115 | 116 | /** 117 | * Gets the value of the column_design property, or Optional.empty() if not set. 118 | * 119 | * @param tableProperties The map of table properties 120 | * @return The column design settings 121 | */ 122 | public static Optional> getColumnDesign( 123 | Map tableProperties) { 124 | requireNonNull(tableProperties); 125 | 126 | @SuppressWarnings("unchecked") 127 | String json = (String) tableProperties.get(COLUMN_DESIGN); 128 | if (json == null) { 129 | return Optional.empty(); 130 | } 131 | 132 | try { 133 | Map map = mapper.readValue(json, new TypeReference>() { 134 | }); 135 | return Optional.of(map); 136 | } catch (IOException e) { 137 | throw new RuntimeException(e); 138 | } 139 | } 140 | 141 | public static Optional getPartitionDesign(Map tableProperties) 142 | { 143 | requireNonNull(tableProperties); 144 | 145 | String json = (String) tableProperties.get(PARTITION_DESIGN); 146 | if (json != null) { 147 | try { 148 | PartitionDesign design = mapper.readValue(json, PartitionDesign.class); 149 | return Optional.of(design); 150 | } 151 | catch (IOException e) { 152 | throw new RuntimeException(e); 153 | } 154 | } else { 155 | return Optional.empty(); 156 | } 157 | } 158 | 159 | public static List getRangePartitions(Map tableProperties) { 160 | requireNonNull(tableProperties); 161 | 162 | @SuppressWarnings("unchecked") 163 | String json = (String) tableProperties.get(RANGE_PARTITIONS); 164 | if (json != null) { 165 | try { 166 | RangePartition[] partitions = mapper.readValue(json, RangePartition[].class); 167 | if (partitions == null) { 168 | return ImmutableList.of(); 169 | } else { 170 | return ImmutableList.copyOf(partitions); 171 | } 172 | } catch (IOException e) { 173 | throw new RuntimeException(e); 174 | } 175 | } else { 176 | return ImmutableList.of(); 177 | } 178 | } 179 | 180 | public static RangePartition parseRangePartition(String json) { 181 | if (json == null) { 182 | return null; 183 | } else { 184 | try { 185 | return mapper.readValue(json, RangePartition.class); 186 | } catch (IOException e) { 187 | throw new RuntimeException(e); 188 | } 189 | } 190 | } 191 | 192 | public static Optional getNumReplicas(Map tableProperties) { 193 | requireNonNull(tableProperties); 194 | 195 | @SuppressWarnings("unchecked") 196 | Integer numReplicas = (Integer) tableProperties.get(NUM_REPLICAS); 197 | return Optional.ofNullable(numReplicas); 198 | } 199 | 200 | public static Map toMap(KuduTable table) { 201 | Map properties = new HashMap<>(); 202 | 203 | LinkedHashMap columns = getColumns(table); 204 | 205 | PartitionDesign partitionDesign = getPartitionDesign(table); 206 | 207 | List rangePartitionList = getRangePartitionList(table, DEFAULT_DEADLINE); 208 | 209 | try { 210 | String columnDesignValue = mapper.writeValueAsString(columns); 211 | properties.put(COLUMN_DESIGN, columnDesignValue); 212 | 213 | String partitionDesignValue = mapper.writeValueAsString(partitionDesign); 214 | properties.put(PARTITION_DESIGN, partitionDesignValue); 215 | 216 | String partitionRangesValue = mapper.writeValueAsString(rangePartitionList); 217 | properties.put(RANGE_PARTITIONS, partitionRangesValue); 218 | 219 | properties.put(NUM_REPLICAS, table.getNumReplicas()); 220 | 221 | return properties; 222 | } catch (IOException e) { 223 | throw new RuntimeException(e); 224 | } 225 | } 226 | 227 | private static List getRangePartitionList(KuduTable table, long deadline) { 228 | List rangePartitions = new ArrayList(); 229 | if (!table.getPartitionSchema().getRangeSchema().getColumns().isEmpty()) { 230 | try { 231 | Iterator var4 = table.getTabletsLocations(deadline).iterator(); 232 | 233 | while (var4.hasNext()) { 234 | LocatedTablet tablet = (LocatedTablet) var4.next(); 235 | Partition partition = tablet.getPartition(); 236 | if (Iterators.all(partition.getHashBuckets().iterator(), Predicates.equalTo(0))) { 237 | RangePartition rangePartition = buildRangePartition(table, partition); 238 | rangePartitions.add(rangePartition); 239 | } 240 | } 241 | } catch (Exception e) { 242 | throw new RuntimeException(e); 243 | } 244 | } 245 | 246 | return rangePartitions; 247 | } 248 | 249 | private static RangePartition buildRangePartition(KuduTable table, Partition partition) throws Exception { 250 | RangePartition rangePartition = new RangePartition(); 251 | 252 | RangeBoundValue lower = buildRangePartitionBound(table, partition.getRangeKeyStart()); 253 | RangeBoundValue upper = buildRangePartitionBound(table, partition.getRangeKeyEnd()); 254 | rangePartition.setLower(lower); 255 | rangePartition.setUpper(upper); 256 | 257 | return rangePartition; 258 | } 259 | 260 | private static RangeBoundValue buildRangePartitionBound(KuduTable table, byte[] rangeKey) throws Exception { 261 | if (rangeKey.length == 0) { 262 | return null; 263 | } else { 264 | Schema schema = table.getSchema(); 265 | PartitionSchema partitionSchema = table.getPartitionSchema(); 266 | PartitionSchema.RangeSchema rangeSchema = partitionSchema.getRangeSchema(); 267 | List rangeColumns = rangeSchema.getColumns(); 268 | 269 | final int numColumns = rangeColumns.size(); 270 | 271 | PartialRow bound = KeyEncoderAccessor.decodeRangePartitionKey(schema, partitionSchema, rangeKey); 272 | 273 | RangeBoundValue value = new RangeBoundValue(); 274 | ArrayList list = new ArrayList<>(); 275 | for (int i = 0; i < numColumns; i++) { 276 | Object obj = toValue(schema, bound, rangeColumns.get(i)); 277 | list.add(obj); 278 | } 279 | value.setValues(list); 280 | return value; 281 | } 282 | } 283 | 284 | private static Object toValue(Schema schema, PartialRow bound, Integer idx) { 285 | Type type = schema.getColumnByIndex(idx).getType(); 286 | switch (type) { 287 | case UNIXTIME_MICROS: 288 | long millis = bound.getLong(idx) / 1000; 289 | return ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC).print(millis); 290 | case STRING: 291 | return bound.getString(idx); 292 | case INT64: 293 | return bound.getLong(idx); 294 | case INT32: 295 | return bound.getInt(idx); 296 | case INT16: 297 | return bound.getShort(idx); 298 | case INT8: 299 | short s = bound.getByte(idx); 300 | return s; 301 | case BOOL: 302 | return bound.getBoolean(idx); 303 | case BINARY: 304 | return bound.getBinaryCopy(idx); 305 | default: 306 | throw new IllegalStateException("Unhandled type " + type + " for range partition"); 307 | } 308 | } 309 | 310 | private static LinkedHashMap getColumns(KuduTable table) { 311 | Schema schema = table.getSchema(); 312 | LinkedHashMap columns = new LinkedHashMap<>(); 313 | for (ColumnSchema columnSchema : schema.getColumns()) { 314 | ColumnDesign design = new ColumnDesign(); 315 | design.setNullable(columnSchema.isNullable()); 316 | design.setKey(columnSchema.isKey()); 317 | if (columnSchema.getCompressionAlgorithm() != null) { 318 | design.setCompression(columnSchema.getCompressionAlgorithm().name()); 319 | } 320 | if (columnSchema.getEncoding() != null) { 321 | design.setEncoding(columnSchema.getEncoding().name()); 322 | } 323 | columns.put(columnSchema.getName(), design); 324 | } 325 | return columns; 326 | } 327 | 328 | public static PartitionDesign getPartitionDesign(KuduTable table) { 329 | Schema schema = table.getSchema(); 330 | PartitionDesign partitionDesign = new PartitionDesign(); 331 | PartitionSchema partitionSchema = table.getPartitionSchema(); 332 | 333 | List hashPartitions = partitionSchema.getHashBucketSchemas().stream() 334 | .map(hashBucketSchema -> { 335 | HashPartitionDefinition hash = new HashPartitionDefinition(); 336 | List cols = hashBucketSchema.getColumnIds().stream() 337 | .map(idx -> schema.getColumnByIndex(idx).getName()).collect(toImmutableList()); 338 | hash.setColumns(cols); 339 | hash.setBuckets(hashBucketSchema.getNumBuckets()); 340 | return hash; 341 | }).collect(toImmutableList()); 342 | partitionDesign.setHash(hashPartitions); 343 | 344 | List rangeColumns = partitionSchema.getRangeSchema().getColumns(); 345 | if (!rangeColumns.isEmpty()) { 346 | RangePartitionDefinition definition = new RangePartitionDefinition(); 347 | definition.setColumns(rangeColumns.stream() 348 | .map(i -> schema.getColumns().get(i).getName()) 349 | .collect(ImmutableList.toImmutableList())); 350 | partitionDesign.setRange(definition); 351 | } 352 | 353 | return partitionDesign; 354 | } 355 | 356 | public static PartialRow toRangeBoundToPartialRow(Schema schema, RangePartitionDefinition definition, 357 | RangeBoundValue boundValue) { 358 | PartialRow partialRow = new PartialRow(schema); 359 | if (boundValue != null) { 360 | List rangeColumns = definition.getColumns().stream() 361 | .map(name -> schema.getColumnIndex(name)).collect(toImmutableList()); 362 | 363 | if (rangeColumns.size() != boundValue.getValues().size()) { 364 | throw new IllegalStateException("Expected " + rangeColumns.size() 365 | + " range columns, but got " + boundValue.getValues().size()); 366 | } 367 | for (int i = 0; i < rangeColumns.size(); i++) { 368 | Object obj = boundValue.getValues().get(i); 369 | int idx = rangeColumns.get(i); 370 | ColumnSchema columnSchema = schema.getColumnByIndex(idx); 371 | setColumnValue(partialRow, idx, obj, columnSchema.getType(), columnSchema.getName()); 372 | } 373 | } 374 | return partialRow; 375 | } 376 | 377 | private static void setColumnValue(PartialRow partialRow, int idx, Object obj, Type type, String name) { 378 | Number n; 379 | 380 | switch (type) { 381 | case STRING: 382 | if (obj instanceof String) { 383 | partialRow.addString(idx, (String) obj); 384 | } else { 385 | handleInvalidValue(name, type, obj); 386 | } 387 | break; 388 | case INT64: 389 | n = toNumber(obj, type, name); 390 | partialRow.addLong(idx, n.longValue()); 391 | break; 392 | case INT32: 393 | n = toNumber(obj, type, name); 394 | partialRow.addInt(idx, n.intValue()); 395 | break; 396 | case INT16: 397 | n = toNumber(obj, type, name); 398 | partialRow.addShort(idx, n.shortValue()); 399 | break; 400 | case INT8: 401 | n = toNumber(obj, type, name); 402 | partialRow.addByte(idx, n.byteValue()); 403 | break; 404 | case DOUBLE: 405 | n = toNumber(obj, type, name); 406 | partialRow.addDouble(idx, n.doubleValue()); 407 | break; 408 | case FLOAT: 409 | n = toNumber(obj, type, name); 410 | partialRow.addFloat(idx, n.floatValue()); 411 | break; 412 | case UNIXTIME_MICROS: 413 | long l = toUnixTimeMicros(obj, type, name); 414 | partialRow.addLong(idx, l); 415 | break; 416 | case BOOL: 417 | boolean b = toBoolean(obj, type, name); 418 | partialRow.addBoolean(idx, b); 419 | break; 420 | case BINARY: 421 | byte[] bytes = toByteArray(obj, type, name); 422 | partialRow.addBinary(idx, bytes); 423 | break; 424 | default: 425 | handleInvalidValue(name, type, obj); 426 | break; 427 | } 428 | } 429 | 430 | private static byte[] toByteArray(Object obj, Type type, String name) { 431 | if (obj instanceof byte[]) { 432 | return (byte[]) obj; 433 | } else if (obj instanceof String) { 434 | return Base64.getDecoder().decode((String) obj); 435 | } else { 436 | handleInvalidValue(name, type, obj); 437 | return null; 438 | } 439 | } 440 | 441 | private static boolean toBoolean(Object obj, Type type, String name) { 442 | if (obj instanceof Boolean) { 443 | return (Boolean) obj; 444 | } else if (obj instanceof String) { 445 | return Boolean.valueOf((String) obj); 446 | } else { 447 | handleInvalidValue(name, type, obj); 448 | return false; 449 | } 450 | } 451 | 452 | private static long toUnixTimeMicros(Object obj, Type type, String name) { 453 | if (Number.class.isAssignableFrom(obj.getClass())) { 454 | return ((Number) obj).longValue(); 455 | } else if (obj instanceof String) { 456 | String s = (String) obj; 457 | s = s.trim().replace(' ', 'T'); 458 | long millis = ISODateTimeFormat.dateOptionalTimeParser().withZone(DateTimeZone.UTC).parseMillis(s); 459 | return millis * 1000; 460 | } else { 461 | handleInvalidValue(name, type, obj); 462 | return 0; 463 | } 464 | } 465 | 466 | private static Number toNumber(Object obj, Type type, String name) { 467 | if (Number.class.isAssignableFrom(obj.getClass())) { 468 | return (Number) obj; 469 | } else if (obj instanceof String) { 470 | String s = (String) obj; 471 | BigDecimal d = new BigDecimal((String) obj); 472 | return d; 473 | } else { 474 | handleInvalidValue(name, type, obj); 475 | return 0; 476 | } 477 | } 478 | 479 | private static void handleInvalidValue(String name, Type type, Object obj) { 480 | throw new IllegalStateException("Invalid value " + obj + " for column " + name + " of type " + type); 481 | } 482 | } 483 | --------------------------------------------------------------------------------