Skip to content

Commit

Permalink
Replace all toUpperCase and toLowerCase using default locale with exp…
Browse files Browse the repository at this point in the history
…licit Locale.ENGLISH.
  • Loading branch information
Henning Schmiedehausen authored and dain committed Oct 2, 2014
1 parent 78e97fe commit 26b4c2a
Show file tree
Hide file tree
Showing 37 changed files with 115 additions and 71 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import static com.google.common.collect.Iterables.getOnlyElement;
import static com.google.common.collect.Maps.fromProperties;
import static java.util.Collections.nCopies;
import static java.util.Locale.ENGLISH;

public class BaseJdbcClient
implements JdbcClient
Expand Down Expand Up @@ -117,7 +118,7 @@ public Set<String> getSchemaNames()
ResultSet resultSet = connection.getMetaData().getSchemas()) {
ImmutableSet.Builder<String> schemaNames = ImmutableSet.builder();
while (resultSet.next()) {
String schemaName = resultSet.getString("TABLE_SCHEM").toLowerCase();
String schemaName = resultSet.getString("TABLE_SCHEM").toLowerCase(ENGLISH);
// skip internal schemas
if (!schemaName.equals("information_schema")) {
schemaNames.add(schemaName);
Expand All @@ -136,7 +137,7 @@ public List<SchemaTableName> getTableNames(@Nullable String schema)
try (Connection connection = driver.connect(connectionUrl, connectionProperties)) {
DatabaseMetaData metadata = connection.getMetaData();
if (metadata.storesUpperCaseIdentifiers() && (schema != null)) {
schema = schema.toUpperCase();
schema = schema.toUpperCase(ENGLISH);
}
try (ResultSet resultSet = getTables(connection, schema, null)) {
ImmutableList.Builder<SchemaTableName> list = ImmutableList.builder();
Expand All @@ -160,8 +161,8 @@ public JdbcTableHandle getTableHandle(SchemaTableName schemaTableName)
String jdbcSchemaName = schemaTableName.getSchemaName();
String jdbcTableName = schemaTableName.getTableName();
if (metadata.storesUpperCaseIdentifiers()) {
jdbcSchemaName = jdbcSchemaName.toUpperCase();
jdbcTableName = jdbcTableName.toUpperCase();
jdbcSchemaName = jdbcSchemaName.toUpperCase(ENGLISH);
jdbcTableName = jdbcTableName.toUpperCase(ENGLISH);
}
try (ResultSet resultSet = getTables(connection, jdbcSchemaName, jdbcTableName)) {
List<JdbcTableHandle> tableHandles = new ArrayList<>();
Expand Down Expand Up @@ -284,8 +285,8 @@ public JdbcOutputTableHandle beginCreateTable(ConnectorTableMetadata tableMetada
try (Connection connection = driver.connect(connectionUrl, connectionProperties)) {
boolean uppercase = connection.getMetaData().storesUpperCaseIdentifiers();
if (uppercase) {
schema = schema.toUpperCase();
table = table.toUpperCase();
schema = schema.toUpperCase(ENGLISH);
table = table.toUpperCase(ENGLISH);
}
String catalog = connection.getCatalog();

Expand All @@ -300,7 +301,7 @@ public JdbcOutputTableHandle beginCreateTable(ConnectorTableMetadata tableMetada
for (ColumnMetadata column : tableMetadata.getColumns()) {
String columnName = column.getName();
if (uppercase) {
columnName = columnName.toUpperCase();
columnName = columnName.toUpperCase(ENGLISH);
}
columnNames.add(columnName);
columnTypes.add(column.getType());
Expand Down Expand Up @@ -377,8 +378,8 @@ protected SchemaTableName getSchemaTableName(ResultSet resultSet)
throws SQLException
{
return new SchemaTableName(
resultSet.getString("TABLE_SCHEM").toLowerCase(),
resultSet.getString("TABLE_NAME").toLowerCase());
resultSet.getString("TABLE_SCHEM").toLowerCase(ENGLISH),
resultSet.getString("TABLE_NAME").toLowerCase(ENGLISH));
}

protected void execute(Connection connection, String query)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Locale.ENGLISH;
import static org.testng.Assert.assertEquals;

final class MetadataUtil
Expand Down Expand Up @@ -61,7 +62,7 @@ public TestingTypeDeserializer()
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
Type type = types.get(value.toLowerCase());
Type type = types.get(value.toLowerCase(ENGLISH));
checkArgument(type != null, "Unknown type %s", value);
return type;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import static com.facebook.presto.plugin.jdbc.TestingDatabase.CONNECTOR_ID;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static java.util.Locale.ENGLISH;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;
Expand Down Expand Up @@ -63,7 +64,7 @@ public void testMetadata()
SchemaTableName schemaTableName = new SchemaTableName("example", "numbers");
JdbcTableHandle table = jdbcClient.getTableHandle(schemaTableName);
assertNotNull(table, "table is null");
assertEquals(table.getCatalogName(), catalogName.toUpperCase());
assertEquals(table.getCatalogName(), catalogName.toUpperCase(ENGLISH));
assertEquals(table.getSchemaName(), "EXAMPLE");
assertEquals(table.getTableName(), "NUMBERS");
assertEquals(table.getSchemaTableName(), schemaTableName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
import static java.util.Locale.ENGLISH;
import static java.util.concurrent.TimeUnit.MILLISECONDS;

/**
Expand Down Expand Up @@ -238,13 +239,13 @@ public CassandraTableHandle getTableHandle(SchemaTableName schemaTableName)

public String getCaseSensitiveSchemaName(String caseInsensitiveName)
{
String caseSensitiveSchemaName = getCacheValue(schemaNamesCache, "", RuntimeException.class).get(caseInsensitiveName.toLowerCase());
String caseSensitiveSchemaName = getCacheValue(schemaNamesCache, "", RuntimeException.class).get(caseInsensitiveName.toLowerCase(ENGLISH));
return caseSensitiveSchemaName == null ? caseInsensitiveName : caseSensitiveSchemaName;
}

public String getCaseSensitiveTableName(SchemaTableName schemaTableName)
{
String caseSensitiveTableName = getCacheValue(tableNamesCache, schemaTableName.getSchemaName(), SchemaNotFoundException.class).get(schemaTableName.getTableName().toLowerCase());
String caseSensitiveTableName = getCacheValue(tableNamesCache, schemaTableName.getSchemaName(), SchemaNotFoundException.class).get(schemaTableName.getTableName().toLowerCase(ENGLISH));
return caseSensitiveTableName == null ? schemaTableName.getTableName() : caseSensitiveTableName;
}

Expand Down Expand Up @@ -340,7 +341,7 @@ private static Function<String, String> toLowerCase()
@Override
public String apply(String str)
{
return str.toLowerCase();
return str.toLowerCase(ENGLISH);
}
};
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import static com.google.common.collect.Iterables.transform;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.Locale.ENGLISH;

public class CassandraMetadata
implements ConnectorMetadata
Expand Down Expand Up @@ -123,7 +124,7 @@ public List<SchemaTableName> listTables(ConnectorSession session, String schemaN
for (String schemaName : listSchemas(session, schemaNameOrNull)) {
try {
for (String tableName : schemaProvider.getAllTables(schemaName)) {
tableNames.add(new SchemaTableName(schemaName, tableName.toLowerCase()));
tableNames.add(new SchemaTableName(schemaName, tableName.toLowerCase(ENGLISH)));
}
}
catch (SchemaNotFoundException e) {
Expand Down Expand Up @@ -159,7 +160,7 @@ private Map<String, ConnectorColumnHandle> getColumnHandles(ConnectorTableHandle
ImmutableMap.Builder<String, ConnectorColumnHandle> columnHandles = ImmutableMap.builder();
for (CassandraColumnHandle columnHandle : table.getColumns()) {
if (includeSampleWeight || !columnHandle.getName().equals(SAMPLE_WEIGHT_COLUMN_NAME)) {
columnHandles.put(CassandraCqlUtils.cqlNameToSqlName(columnHandle.getName()).toLowerCase(), columnHandle);
columnHandles.put(CassandraCqlUtils.cqlNameToSqlName(columnHandle.getName()).toLowerCase(ENGLISH), columnHandle);
}
}
return columnHandles.build();
Expand Down Expand Up @@ -263,7 +264,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
List<Type> types = columnTypes.build();
StringBuilder queryBuilder = new StringBuilder(String.format("CREATE TABLE \"%s\".\"%s\"(id uuid primary key", schemaName, tableName));
if (tableMetadata.isSampled()) {
queryBuilder.append(", ").append(SAMPLE_WEIGHT_COLUMN_NAME).append(" ").append(BIGINT.name().toLowerCase());
queryBuilder.append(", ").append(SAMPLE_WEIGHT_COLUMN_NAME).append(" ").append(BIGINT.name().toLowerCase(ENGLISH));
columnExtra.add(new ExtraColumnMetadata(SAMPLE_WEIGHT_COLUMN_NAME, true));
}
for (int i = 0; i < columns.size(); i++) {
Expand All @@ -272,7 +273,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
queryBuilder.append(", ")
.append(name)
.append(" ")
.append(toCassandraType(type).name().toLowerCase());
.append(toCassandraType(type).name().toLowerCase(ENGLISH));
}
queryBuilder.append(") ");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
import java.util.List;
import java.util.Set;

import static java.util.Locale.ENGLISH;

public final class CassandraCqlUtils
{
private CassandraCqlUtils()
Expand Down Expand Up @@ -70,11 +72,11 @@ public static String validColumnName(String identifier)

private static String validIdentifier(String identifier)
{
if (!identifier.equals(identifier.toLowerCase())) {
if (!identifier.equals(identifier.toLowerCase(ENGLISH))) {
return quoteIdentifier(identifier);
}

if (keywords.contains(identifier.toUpperCase())) {
if (keywords.contains(identifier.toUpperCase(ENGLISH))) {
return quoteIdentifier(identifier);
}
return identifier;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.google.common.base.Preconditions.checkArgument;
import static io.airlift.testing.Assertions.assertInstanceOf;
import static java.util.Locale.ENGLISH;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
Expand Down Expand Up @@ -146,7 +147,7 @@ public void testGetDatabaseNames()
throws Exception
{
List<String> databases = metadata.listSchemaNames(SESSION);
assertTrue(databases.contains(database.toLowerCase()));
assertTrue(databases.contains(database.toLowerCase(ENGLISH)));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import java.util.Locale;
import java.util.TimeZone;

import static java.util.Locale.ENGLISH;

public class ClientOptions
{
@Option(name = "--server", title = "server", description = "Presto server location (default: localhost:8080)")
Expand Down Expand Up @@ -68,7 +70,7 @@ public ClientSession toClientSession()

public static URI parseServer(String server)
{
server = server.toLowerCase();
server = server.toLowerCase(ENGLISH);
if (server.startsWith("http:https://") || server.startsWith("https://")) {
return URI.create(server);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import static com.google.common.io.ByteStreams.nullOutputStream;
import static io.airlift.log.Logging.Level;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;
import static jline.internal.Configuration.getUserHome;

@Command(name = "presto", description = "Presto interactive console")
Expand Down Expand Up @@ -142,7 +143,7 @@ private static void runConsole(QueryRunner queryRunner, ClientSession session)
if (command.endsWith(";")) {
command = command.substring(0, command.length() - 1).trim();
}
switch (command.toLowerCase()) {
switch (command.toLowerCase(ENGLISH)) {
case "exit":
case "quit":
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static io.airlift.json.JsonCodec.listJsonCodec;
import static java.util.Locale.ENGLISH;

public final class MetadataUtil
{
Expand Down Expand Up @@ -67,7 +68,7 @@ public TestingTypeDeserializer()
@Override
protected Type _deserialize(String value, DeserializationContext context)
{
Type type = types.get(value.toLowerCase());
Type type = types.get(value.toLowerCase(ENGLISH));
if (type == null) {
throw new IllegalArgumentException(String.valueOf("Unknown type " + value));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@
import static io.airlift.slice.Slices.utf8Slice;
import static io.airlift.testing.Assertions.assertInstanceOf;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Locale.ENGLISH;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -1532,7 +1533,7 @@ private static ImmutableMap<String, Integer> indexColumns(List<ConnectorColumnHa

private static String randomName()
{
return UUID.randomUUID().toString().toLowerCase().replace("-", "");
return UUID.randomUUID().toString().toLowerCase(ENGLISH).replace("-", "");
}

private static Function<ColumnMetadata, String> columnNameGetter()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import static com.google.common.collect.Iterables.getOnlyElement;
import static com.google.common.util.concurrent.MoreExecutors.sameThreadExecutor;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Locale.ENGLISH;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -104,7 +105,7 @@ protected void setupHive(String databaseName)
database = databaseName;
tableS3 = new SchemaTableName(database, "presto_test_s3");

String random = UUID.randomUUID().toString().toLowerCase().replace("-", "");
String random = UUID.randomUUID().toString().toLowerCase(ENGLISH).replace("-", "");
temporaryCreateTable = new SchemaTableName(database, "tmp_presto_test_create_s3_" + random);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import static com.google.common.collect.Iterators.concat;
import static com.google.common.collect.Iterators.transform;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;

public class PrestoResultSet
implements ResultSet
Expand Down Expand Up @@ -1690,7 +1691,7 @@ private int columnIndex(String label)
if (label == null) {
throw new SQLException("Column label is null");
}
Integer index = fieldMap.get(label.toLowerCase());
Integer index = fieldMap.get(label.toLowerCase(ENGLISH));
if (index == null) {
throw new SQLException("Invalid column label: " + label);
}
Expand Down Expand Up @@ -1768,7 +1769,7 @@ private static Map<String, Integer> getFieldMap(List<Column> columns)
{
Map<String, Integer> map = new HashMap<>();
for (int i = 0; i < columns.size(); i++) {
String name = columns.get(i).getName().toLowerCase();
String name = columns.get(i).getName().toLowerCase(ENGLISH);
if (!map.containsKey(name)) {
map.put(name, i + 1);
}
Expand All @@ -1786,7 +1787,7 @@ private static List<ColumnInfo> getColumnInfo(List<Column> columns)
.setTableName("") // TODO
.setColumnLabel(column.getName())
.setColumnName(column.getName()) // TODO
.setColumnTypeName(column.getType().toUpperCase())
.setColumnTypeName(column.getType().toUpperCase(ENGLISH))
.setNullable(ResultSetMetaData.columnNullableUnknown)
.setCurrency(false);
setTypeInfo(builder, column.getType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,13 +96,13 @@ private static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClie
{
long start = System.nanoTime();
log.info("Running import for %s", table.getTableName());
TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedTableName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase()));
TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedTableName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)));
log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit());
}

private static String kafkaTopicName(TpchTable<?> table)
{
return TPCH_SCHEMA + "." + table.getTableName().toLowerCase();
return TPCH_SCHEMA + "." + table.getTableName().toLowerCase(ENGLISH);
}

private static Map<SchemaTableName, KafkaTopicDescription> createTpchTopicDescriptions(Metadata metadata, Iterable<TpchTable<?>> tables)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import java.util.EnumSet;

import static java.util.Locale.ENGLISH;
import static org.objectweb.asm.Opcodes.ACC_ABSTRACT;
import static org.objectweb.asm.Opcodes.ACC_ANNOTATION;
import static org.objectweb.asm.Opcodes.ACC_BRIDGE;
Expand Down Expand Up @@ -72,7 +73,7 @@ public int getModifier()
@Override
public String toString()
{
return super.name().toLowerCase();
return super.name().toLowerCase(ENGLISH);
}

public static EnumSet<Access> a(Access... access)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.util.Types.checkType;
import static com.google.common.base.Preconditions.checkNotNull;
import static java.util.Locale.ENGLISH;
import static javax.management.ObjectName.WILDCARD;

public class JmxMetadata
Expand Down Expand Up @@ -114,7 +115,7 @@ public List<SchemaTableName> listTables(ConnectorSession session, String schemaN
Builder<SchemaTableName> tableNames = ImmutableList.builder();
for (ObjectName objectName : mbeanServer.queryNames(WILDCARD, null)) {
// todo remove lower case when presto supports mixed case names
tableNames.add(new SchemaTableName(SCHEMA_NAME, objectName.toString().toLowerCase()));
tableNames.add(new SchemaTableName(SCHEMA_NAME, objectName.toString().toLowerCase(ENGLISH)));
}
return tableNames.build();
}
Expand All @@ -134,7 +135,7 @@ public Map<String, ConnectorColumnHandle> getColumnHandles(ConnectorTableHandle
@Override
public String apply(JmxColumnHandle input)
{
return input.getColumnName().toLowerCase();
return input.getColumnName().toLowerCase(ENGLISH);
}
}));
}
Expand Down
Loading

0 comments on commit 26b4c2a

Please sign in to comment.