Skip to content

Commit

Permalink
Fix failure when symbol name ends in colon
Browse files Browse the repository at this point in the history
  • Loading branch information
martint committed Apr 23, 2024
1 parent 9962596 commit 3c0fbd0
Show file tree
Hide file tree
Showing 3 changed files with 105 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,15 @@
import io.trino.spi.type.TypeId;
import io.trino.spi.type.TypeManager;

import java.util.Base64;

import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;

public class SymbolKeyDeserializer
extends KeyDeserializer
{
private static final Base64.Decoder DECODER = Base64.getDecoder();
private final TypeManager typeManager;

@Inject
Expand All @@ -33,7 +39,11 @@ public SymbolKeyDeserializer(TypeManager typeManager)
@Override
public Object deserializeKey(String key, DeserializationContext context)
{
String[] parts = key.split("::");
return new Symbol(typeManager.getType(TypeId.of(parts[1])), parts[0]);
String[] parts = key.split(":");
checkArgument(parts.length == 2, "Expected two parts, found: " + parts.length);

String name = new String(DECODER.decode(parts[0].getBytes(UTF_8)), UTF_8);
String type = new String(DECODER.decode(parts[1].getBytes(UTF_8)), UTF_8);
return new Symbol(typeManager.getType(TypeId.of(type)), name);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,21 @@
import com.fasterxml.jackson.databind.SerializerProvider;

import java.io.IOException;
import java.util.Base64;

import static java.nio.charset.StandardCharsets.UTF_8;

public class SymbolKeySerializer
extends JsonSerializer<Symbol>
{
private static final Base64.Encoder ENCODER = Base64.getEncoder();

@Override
public void serialize(Symbol value, JsonGenerator gen, SerializerProvider serializers)
public void serialize(Symbol value, JsonGenerator generator, SerializerProvider serializers)
throws IOException
{
gen.writeFieldName(value.name() + "::" + value.type().getTypeId().getId());
String name = ENCODER.encodeToString(value.name().getBytes(UTF_8));
String type = ENCODER.encodeToString(value.type().getTypeId().getId().getBytes(UTF_8));
generator.writeFieldName(name + ":" + type);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* Licensed 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 io.trino.sql.planner;

import com.fasterxml.jackson.databind.KeyDeserializer;
import com.google.common.collect.ImmutableMap;
import com.google.common.reflect.TypeToken;
import io.airlift.json.JsonCodec;
import io.airlift.json.JsonCodecFactory;
import io.airlift.json.ObjectMapperProvider;
import io.trino.spi.type.Type;
import io.trino.type.TypeDeserializer;
import org.junit.jupiter.api.Test;

import java.util.Map;

import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.RowType.field;
import static io.trino.spi.type.RowType.rowType;
import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER;
import static org.assertj.core.api.Assertions.assertThat;

class TestSymbolSerializer
{
private static final JsonCodecFactory CODEC_FACTORY;

static {
ObjectMapperProvider provider = new ObjectMapperProvider();
provider.setKeyDeserializers(ImmutableMap.<Class<?>, KeyDeserializer>builder()
.put(Symbol.class, new SymbolKeyDeserializer(TESTING_TYPE_MANAGER))
.buildOrThrow());

provider.setJsonDeserializers(ImmutableMap.of(
Type.class, new TypeDeserializer(TESTING_TYPE_MANAGER)));

CODEC_FACTORY = new JsonCodecFactory(provider);
}

private static final JsonCodec<Symbol> SYMBOL_CODEC = CODEC_FACTORY.jsonCodec(Symbol.class);
private static final JsonCodec<Map<Symbol, String>> MAP_CODEC = CODEC_FACTORY.jsonCodec(new TypeToken<>() { });

@Test
void testAsValue()
{
assertThat(SYMBOL_CODEC.fromJson(SYMBOL_CODEC.toJson(new Symbol(BIGINT, "abc123!@# :/{}[]*"))))
.isEqualTo(new Symbol(BIGINT, "abc123!@# :/{}[]*"));

assertThat(SYMBOL_CODEC.fromJson(SYMBOL_CODEC.toJson(new Symbol(BIGINT, "abc::def"))))
.isEqualTo(new Symbol(BIGINT, "abc::def"));

assertThat(SYMBOL_CODEC.fromJson(SYMBOL_CODEC.toJson(new Symbol(BIGINT, "a:"))))
.isEqualTo(new Symbol(BIGINT, "a:"));

assertThat(SYMBOL_CODEC.fromJson(SYMBOL_CODEC.toJson(new Symbol(rowType(field("abc", BIGINT)), "a:"))))
.isEqualTo(new Symbol(rowType(field("abc", BIGINT)), "a:"));

assertThat(SYMBOL_CODEC.fromJson(SYMBOL_CODEC.toJson(new Symbol(rowType(field("abc::123", BIGINT)), "a:"))))
.isEqualTo(new Symbol(rowType(field("abc::123", BIGINT)), "a:"));
}

@Test
void testAsKey()
{
assertThat(MAP_CODEC.fromJson(MAP_CODEC.toJson(ImmutableMap.of(new Symbol(BIGINT, "abc123!@# :/{}[]*"), ""))))
.isEqualTo(ImmutableMap.of(new Symbol(BIGINT, "abc123!@# :/{}[]*"), ""));

assertThat(MAP_CODEC.fromJson(MAP_CODEC.toJson(ImmutableMap.of(new Symbol(BIGINT, "a:"), ""))))
.isEqualTo(ImmutableMap.of(new Symbol(BIGINT, "a:"), ""));

assertThat(MAP_CODEC.fromJson(MAP_CODEC.toJson(ImmutableMap.of(new Symbol(rowType(field("abc::123", BIGINT)), "a:"), ""))))
.isEqualTo(ImmutableMap.of(new Symbol(rowType(field("abc::123", BIGINT)), "a:"), ""));
}
}

0 comments on commit 3c0fbd0

Please sign in to comment.