diff --git a/pom.xml b/pom.xml index 05b921635949c..5b820bb6d7fc8 100644 --- a/pom.xml +++ b/pom.xml @@ -136,6 +136,7 @@ presto-server-main presto-server-rpm presto-session-property-managers + presto-snowflake presto-spi presto-sqlserver presto-teradata-functions diff --git a/presto-docs/src/main/sphinx/connector.rst b/presto-docs/src/main/sphinx/connector.rst index 76a76872cc33f..d0f9f87e169b4 100644 --- a/presto-docs/src/main/sphinx/connector.rst +++ b/presto-docs/src/main/sphinx/connector.rst @@ -33,6 +33,7 @@ from different data sources. Prometheus Redis Redshift + Snowflake SQL Server System Thrift diff --git a/presto-docs/src/main/sphinx/connector/snowflake.rst b/presto-docs/src/main/sphinx/connector/snowflake.rst new file mode 100644 index 0000000000000..0827cd6819a53 --- /dev/null +++ b/presto-docs/src/main/sphinx/connector/snowflake.rst @@ -0,0 +1,42 @@ +=================== +Snowflake Connector +=================== + +The Snowflake connector allows querying and creating tables in an +external Snowflake account. This can be used to join data between +different systems like Snowflake and Hive, or between two different +Snowflake accounts. + +Configuration +------------- + +To configure the Snowflake connector, create a catalog properties file +in ``etc/catalog`` named, for example, ``snowflake.properties``, to +mount the Snowflake connector as the ``snowflake`` catalog. +Create the file with the following contents, replacing the +connection properties as appropriate for your setup: + +.. code-block:: none + + connector.name=snowflake + connection-url=jdbc:snowflake://.snowflakecomputing.com + connection-user=root + connection-password=secret + snowflake.account=account + snowflake.database=database + snowflake.role=role + snowflake.warehouse=warehouse + + +Multiple Snowflake Databases or Accounts +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +The Snowflake connector can only access a single database within +a Snowflake account. Thus, if you have multiple Snowflake databases, +or want to connect to multiple Snowflake accounts, you must configure +multiple instances of the Snowflake connector. + +To add another catalog, simply add another properties file to ``etc/catalog`` +with a different name, making sure it ends in ``.properties``. For example, +if you name the property file ``sales.properties``, Presto creates a +catalog named ``sales`` using the configured connector. diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 1905ff92b5764..00694e965b7a9 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -253,4 +253,10 @@ + + + + + + diff --git a/presto-snowflake/pom.xml b/presto-snowflake/pom.xml new file mode 100644 index 0000000000000..8b2241bf5fb53 --- /dev/null +++ b/presto-snowflake/pom.xml @@ -0,0 +1,90 @@ + + + 4.0.0 + + + io.prestosql + presto-root + 345-SNAPSHOT + + + presto-snowflake + Presto - Snowflake Connector + presto-plugin + + + ${project.parent.basedir} + + + + + io.prestosql + presto-base-jdbc + + + + io.airlift + configuration + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.inject + javax.inject + + + + net.snowflake + snowflake-jdbc + 3.10.0 + + + + + io.prestosql + presto-spi + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + + + io.prestosql + presto-main + test + + + + org.testng + testng + test + + + diff --git a/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClient.java b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClient.java new file mode 100644 index 0000000000000..3702292755e7a --- /dev/null +++ b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClient.java @@ -0,0 +1,72 @@ +/* + * 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.prestosql.plugin.snowflake; + +import io.prestosql.plugin.jdbc.BaseJdbcClient; +import io.prestosql.plugin.jdbc.BaseJdbcConfig; +import io.prestosql.plugin.jdbc.ConnectionFactory; +import io.prestosql.plugin.jdbc.JdbcIdentity; +import io.prestosql.plugin.jdbc.JdbcSplit; +import io.prestosql.plugin.jdbc.WriteMapping; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.Type; + +import javax.inject.Inject; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.Optional; +import java.util.function.BiFunction; + +import static io.prestosql.plugin.jdbc.StandardColumnMappings.timestampWriteFunctionUsingSqlTimestamp; +import static io.prestosql.spi.type.TimestampType.TIMESTAMP_NANOS; + +public class SnowflakeClient + extends BaseJdbcClient +{ + @Inject + public SnowflakeClient(BaseJdbcConfig config, ConnectionFactory connectionFactory) + { + super(config, "\"", connectionFactory); + } + + @Override + public boolean isLimitGuaranteed(ConnectorSession session) + { + return true; + } + + @Override + protected Optional> limitFunction() + { + return Optional.of((sql, limit) -> sql + " LIMIT " + limit); + } + + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + if (TIMESTAMP_NANOS.equals(type)) { + return WriteMapping.longMapping("timestamp", timestampWriteFunctionUsingSqlTimestamp(TIMESTAMP_NANOS)); + } + + return super.toWriteMapping(session, type); + } + + // Required since SnowflakeConnectionV1 does not support setReadOnly + @Override + public Connection getConnection(JdbcIdentity identity, JdbcSplit split) throws SQLException + { + return connectionFactory.openConnection(identity); + } +} diff --git a/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClientModule.java b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClientModule.java new file mode 100644 index 0000000000000..e0f5912e28202 --- /dev/null +++ b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeClientModule.java @@ -0,0 +1,59 @@ +/* + * 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.prestosql.plugin.snowflake; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import io.prestosql.plugin.jdbc.BaseJdbcConfig; +import io.prestosql.plugin.jdbc.ConnectionFactory; +import io.prestosql.plugin.jdbc.DriverConnectionFactory; +import io.prestosql.plugin.jdbc.ForBaseJdbc; +import io.prestosql.plugin.jdbc.JdbcClient; +import io.prestosql.plugin.jdbc.credential.CredentialProvider; +import net.snowflake.client.jdbc.SnowflakeDriver; + +import java.util.Properties; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class SnowflakeClientModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(Key.get(JdbcClient.class, ForBaseJdbc.class)) + .to(SnowflakeClient.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(SnowflakeConfig.class); + } + + @Singleton + @Provides + @ForBaseJdbc + public ConnectionFactory getConnectionFactory(BaseJdbcConfig baseJdbcConfig, SnowflakeConfig snowflakeConfig, CredentialProvider credentialProvider) + { + Properties properties = new Properties(); + snowflakeConfig.getAccount().ifPresent(account -> properties.setProperty("account", account)); + snowflakeConfig.getDatabase().ifPresent(database -> properties.setProperty("db", database)); + snowflakeConfig.getRole().ifPresent(role -> properties.setProperty("role", role)); + snowflakeConfig.getWarehouse().ifPresent(warehouse -> properties.setProperty("warehouse", warehouse)); + + return new DriverConnectionFactory(new SnowflakeDriver(), baseJdbcConfig.getConnectionUrl(), properties, credentialProvider); + } +} diff --git a/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeConfig.java b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeConfig.java new file mode 100644 index 0000000000000..f6bca68c65d81 --- /dev/null +++ b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakeConfig.java @@ -0,0 +1,74 @@ +/* + * 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.prestosql.plugin.snowflake; + +import io.airlift.configuration.Config; + +import java.util.Optional; + +public class SnowflakeConfig +{ + private String account; + private String database; + private String role; + private String warehouse; + + public Optional getAccount() + { + return Optional.ofNullable(account); + } + + @Config("snowflake.account") + public SnowflakeConfig setAccount(String account) + { + this.account = account; + return this; + } + + public Optional getDatabase() + { + return Optional.ofNullable(database); + } + + @Config("snowflake.database") + public SnowflakeConfig setDatabase(String database) + { + this.database = database; + return this; + } + + public Optional getRole() + { + return Optional.ofNullable(role); + } + + @Config("snowflake.role") + public SnowflakeConfig setRole(String role) + { + this.role = role; + return this; + } + + public Optional getWarehouse() + { + return Optional.ofNullable(warehouse); + } + + @Config("snowflake.warehouse") + public SnowflakeConfig setWarehouse(String warehouse) + { + this.warehouse = warehouse; + return this; + } +} diff --git a/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakePlugin.java b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakePlugin.java new file mode 100644 index 0000000000000..f34101d917b3e --- /dev/null +++ b/presto-snowflake/src/main/java/io/prestosql/plugin/snowflake/SnowflakePlugin.java @@ -0,0 +1,25 @@ +/* + * 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.prestosql.plugin.snowflake; + +import io.prestosql.plugin.jdbc.JdbcPlugin; + +public class SnowflakePlugin + extends JdbcPlugin +{ + public SnowflakePlugin() + { + super("snowflake", new SnowflakeClientModule()); + } +} diff --git a/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakeConfig.java b/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakeConfig.java new file mode 100644 index 0000000000000..233a017d8bfa2 --- /dev/null +++ b/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakeConfig.java @@ -0,0 +1,55 @@ +/* + * 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.prestosql.plugin.snowflake; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestSnowflakeConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(SnowflakeConfig.class) + .setAccount(null) + .setDatabase(null) + .setRole(null) + .setWarehouse(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("snowflake.account", "MYACCOUNT") + .put("snowflake.database", "MYDATABASE") + .put("snowflake.role", "MYROLE") + .put("snowflake.warehouse", "MYWAREHOUSE") + .build(); + + SnowflakeConfig expected = new SnowflakeConfig() + .setAccount("MYACCOUNT") + .setDatabase("MYDATABASE") + .setRole("MYROLE") + .setWarehouse("MYWAREHOUSE"); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakePlugin.java b/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakePlugin.java new file mode 100644 index 0000000000000..3c8b3de285023 --- /dev/null +++ b/presto-snowflake/src/test/java/io/prestosql/plugin/snowflake/TestSnowflakePlugin.java @@ -0,0 +1,33 @@ +/* + * 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.prestosql.plugin.snowflake; + +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestSnowflakePlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new SnowflakePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("connection-url", "test"), new TestingConnectorContext()); + } +}