Skip to content

Commit

Permalink
Pulsar SQL supports pulsar's primitive schema (apache#4728)
Browse files Browse the repository at this point in the history
### Motivation
Continue the PR of apache#4151
  • Loading branch information
congbobo184 authored and sijie committed Jul 24, 2019
1 parent 0362944 commit 1ab35b0
Show file tree
Hide file tree
Showing 10 changed files with 442 additions and 26 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -175,4 +175,46 @@ public static SchemaType valueOf(int value) {
default: return NONE;
}
}


public boolean isPrimitive() {
return isPrimitiveType(this);
}

public boolean isStruct() {
return isStructType(this);
}

public static boolean isPrimitiveType(SchemaType type) {
switch (type) {
case STRING:
case BOOLEAN:
case INT8:
case INT16:
case INT32:
case INT64:
case FLOAT:
case DOUBLE:
case DATE:
case TIME:
case TIMESTAMP:
case BYTES:
case NONE:
return true;
default:
return false;
}

}

public static boolean isStructType(SchemaType type) {
switch (type) {
case AVRO:
case JSON:
case PROTOBUF:
return true;
default:
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,14 @@
import com.facebook.presto.spi.connector.ConnectorMetadata;
import com.facebook.presto.spi.type.BigintType;
import com.facebook.presto.spi.type.BooleanType;
import com.facebook.presto.spi.type.DateType;
import com.facebook.presto.spi.type.DoubleType;
import com.facebook.presto.spi.type.IntegerType;
import com.facebook.presto.spi.type.RealType;
import com.facebook.presto.spi.type.SmallintType;
import com.facebook.presto.spi.type.TimeType;
import com.facebook.presto.spi.type.TimestampType;
import com.facebook.presto.spi.type.TinyintType;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.VarbinaryType;
import com.facebook.presto.spi.type.VarcharType;
Expand All @@ -55,6 +60,7 @@
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;

import javax.inject.Inject;
import java.util.HashMap;
Expand Down Expand Up @@ -296,6 +302,56 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName,
+ String.format("%s/%s", namespace, schemaTableName.getTableName())
+ ": " + ExceptionUtils.getRootCause(e).getLocalizedMessage(), e);
}
List<ColumnMetadata> handles = getPulsarColumns(
topicName, schemaInfo, withInternalColumns
);


return new ConnectorTableMetadata(schemaTableName, handles);
}

/**
* Convert pulsar schema into presto table metadata.
*/
static List<ColumnMetadata> getPulsarColumns(TopicName topicName,
SchemaInfo schemaInfo,
boolean withInternalColumns) {
SchemaType schemaType = schemaInfo.getType();
if (schemaType.isStruct()) {
return getPulsarColumnsFromStructSchema(topicName, schemaInfo, withInternalColumns);
} else if (schemaType.isPrimitive()) {
return getPulsarColumnsFromPrimitiveSchema(topicName, schemaInfo, withInternalColumns);
} else {
throw new IllegalArgumentException("Unsupported schema : " + schemaInfo);
}
}

static List<ColumnMetadata> getPulsarColumnsFromPrimitiveSchema(TopicName topicName,
SchemaInfo schemaInfo,
boolean withInternalColumns) {
ImmutableList.Builder<ColumnMetadata> builder = ImmutableList.builder();

ColumnMetadata valueColumn = new PulsarColumnMetadata(
"__value__",
convertPulsarType(schemaInfo.getType()),
null, null, false, false,
new String[0],
new Integer[0]);

builder.add(valueColumn);

if (withInternalColumns) {
PulsarInternalColumn.getInternalFields()
.stream()
.forEach(pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false)));
}

return builder.build();
}

static List<ColumnMetadata> getPulsarColumnsFromStructSchema(TopicName topicName,
SchemaInfo schemaInfo,
boolean withInternalColumns) {

String schemaJson = new String(schemaInfo.getSchema());
if (StringUtils.isBlank(schemaJson)) {
Expand All @@ -315,11 +371,44 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName,
builder.addAll(getColumns(null, schema, new HashSet<>(), new Stack<>(), new Stack<>()));

if (withInternalColumns) {
PulsarInternalColumn.getInternalFields().forEach(
pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false)));
PulsarInternalColumn.getInternalFields()
.stream()
.forEach(pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false)));
}
return builder.build();
}
@VisibleForTesting
static Type convertPulsarType(SchemaType pulsarType) {
switch (pulsarType) {
case BOOLEAN:
return BooleanType.BOOLEAN;
case INT8:
return TinyintType.TINYINT;
case INT16:
return SmallintType.SMALLINT;
case INT32:
return IntegerType.INTEGER;
case INT64:
return BigintType.BIGINT;
case FLOAT:
return RealType.REAL;
case DOUBLE:
return DoubleType.DOUBLE;
case NONE:
case BYTES:
return VarbinaryType.VARBINARY;
case STRING:
return VarcharType.VARCHAR;
case DATE:
return DateType.DATE;
case TIME:
return TimeType.TIME;
case TIMESTAMP:
return TimestampType.TIMESTAMP;
default:
log.error("Cannot convert type: %s", pulsarType);
return null;
}

return new ConnectorTableMetadata(schemaTableName, builder.build());
}


Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.sql.presto;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufUtil;
import java.sql.Time;
import java.sql.Timestamp;
import java.util.Date;

import io.netty.util.concurrent.FastThreadLocal;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.schema.AutoConsumeSchema;
import org.apache.pulsar.common.schema.SchemaInfo;

/**
* A presto schema handler that interprets data using pulsar schema.
*/
public class PulsarPrimitiveSchemaHandler implements SchemaHandler {

private final SchemaInfo schemaInfo;
private final Schema<?> schema;

public PulsarPrimitiveSchemaHandler(SchemaInfo schemaInfo) {
this.schemaInfo = schemaInfo;
this.schema = AutoConsumeSchema.getSchema(schemaInfo);
}

@Override
public Object deserialize(ByteBuf byteBuf) {
byte[] data = ByteBufUtil.getBytes(byteBuf);
Object currentRecord = schema.decode(data);
switch (schemaInfo.getType()) {
case DATE:
return ((Date) currentRecord).getTime();
case TIME:
return ((Time) currentRecord).getTime();
case TIMESTAMP:
return ((Timestamp) currentRecord).getTime();
default:
return currentRecord;
}
}

@Override
public Object extractField(int index, Object currentRecord) {
return currentRecord;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -140,9 +140,10 @@ private void initialize(List<PulsarColumnHandle> columnHandles, PulsarSplit puls
this.readOffloaded = pulsarConnectorConfig.getManagedLedgerOffloadDriver() != null;
this.pulsarConnectorConfig = pulsarConnectorConfig;

Schema schema = PulsarConnectorUtils.parseSchema(pulsarSplit.getSchema());

this.schemaHandler = getSchemaHandler(schema, pulsarSplit.getSchemaType(), columnHandles);
this.schemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(
pulsarSplit.getSchemaInfo(),
columnHandles
);

log.info("Initializing split with parameters: %s", pulsarSplit);

Expand All @@ -156,22 +157,6 @@ private void initialize(List<PulsarColumnHandle> columnHandles, PulsarSplit puls
}
}

private SchemaHandler getSchemaHandler(Schema schema, SchemaType schemaType,
List<PulsarColumnHandle> columnHandles) {
SchemaHandler schemaHandler;
switch (schemaType) {
case JSON:
schemaHandler = new JSONSchemaHandler(columnHandles);
break;
case AVRO:
schemaHandler = new AvroSchemaHandler(schema, columnHandles);
break;
default:
throw new PrestoException(NOT_SUPPORTED, "Not supported schema type: " + schemaType);
}
return schemaHandler;
}

private ReadOnlyCursor getCursor(TopicName topicName, Position startPosition, ManagedLedgerFactory
managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig)
throws ManagedLedgerException, InterruptedException {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.sql.presto;

import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.nio.charset.StandardCharsets.UTF_8;

import com.facebook.presto.spi.PrestoException;
import java.util.List;
import org.apache.pulsar.common.schema.SchemaInfo;

class PulsarSchemaHandlers {

static SchemaHandler newPulsarSchemaHandler(SchemaInfo schemaInfo,
List<PulsarColumnHandle> columnHandles) {
if (schemaInfo.getType().isPrimitive()) {
return new PulsarPrimitiveSchemaHandler(schemaInfo);
} else if (schemaInfo.getType().isStruct()) {
switch (schemaInfo.getType()) {
case JSON:
return new JSONSchemaHandler(columnHandles);
case AVRO:
return new AvroSchemaHandler(PulsarConnectorUtils
.parseSchema(new String(schemaInfo.getSchema(), UTF_8)
), columnHandles);
default:
throw new PrestoException(NOT_SUPPORTED, "Not supported schema type: " + schemaInfo.getType());
}

} else {
throw new PrestoException(
NOT_SUPPORTED,
"Schema `" + schemaInfo.getType() + "` is not supported by presto yet : " + schemaInfo);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,11 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;

import java.util.List;
import java.util.Map;

import static java.util.Objects.requireNonNull;

Expand All @@ -46,6 +48,7 @@ public class PulsarSplit implements ConnectorSplit {
private final long startPositionLedgerId;
private final long endPositionLedgerId;
private final TupleDomain<ColumnHandle> tupleDomain;
private final SchemaInfo schemaInfo;

private final PositionImpl startPosition;
private final PositionImpl endPosition;
Expand All @@ -63,8 +66,16 @@ public PulsarSplit(
@JsonProperty("endPositionEntryId") long endPositionEntryId,
@JsonProperty("startPositionLedgerId") long startPositionLedgerId,
@JsonProperty("endPositionLedgerId") long endPositionLedgerId,
@JsonProperty("tupleDomain") TupleDomain<ColumnHandle> tupleDomain) {
@JsonProperty("tupleDomain") TupleDomain<ColumnHandle> tupleDomain,
@JsonProperty("properties") Map<String, String> schemaInfoProperties) {
this.splitId = splitId;
requireNonNull(schemaName, "schema name is null");
this.schemaInfo = SchemaInfo.builder()
.type(schemaType)
.name(schemaName)
.schema(schema.getBytes())
.properties(schemaInfoProperties)
.build();
this.schemaName = requireNonNull(schemaName, "schema name is null");
this.connectorId = requireNonNull(connectorId, "connector id is null");
this.tableName = requireNonNull(tableName, "table name is null");
Expand Down Expand Up @@ -179,4 +190,8 @@ public String toString() {
", endPositionLedgerId=" + endPositionLedgerId +
'}';
}

public SchemaInfo getSchemaInfo() {
return schemaInfo;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,8 @@ Collection<PulsarSplit> getSplitsForTopic(String topicNamePersistenceEncoding,
endPosition.getEntryId(),
startPosition.getLedgerId(),
endPosition.getLedgerId(),
tupleDomain));
tupleDomain,
schemaInfo.getProperties()));
}
return splits;
} finally {
Expand Down
Loading

0 comments on commit 1ab35b0

Please sign in to comment.