diff --git a/core/src/main/java/google/registry/beam/initsql/BackupPaths.java b/core/src/main/java/google/registry/beam/initsql/BackupPaths.java index 52973eb29..33916be85 100644 --- a/core/src/main/java/google/registry/beam/initsql/BackupPaths.java +++ b/core/src/main/java/google/registry/beam/initsql/BackupPaths.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Strings.isNullOrEmpty; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Streams; import org.joda.time.DateTime; @@ -36,6 +37,23 @@ public final class BackupPaths { public static final String COMMIT_LOG_NAME_PREFIX = "commit_diff_until_"; private static final String COMMIT_LOG_PATTERN_TEMPLATE = "%s/" + COMMIT_LOG_NAME_PREFIX + "*"; + /** + * Pattern of the per-project file with Cloud SQL connection information. To get a concrete path, + * user needs to provide the name of the environment, alpha, crash, sandbox, or production. This + * file is meant for applications without access to secrets stored in Datastore. + * + *

In production, this is an base-64 encoded encrypted file with one line, which contains + * space-separated values of Cloud SQL instance name, login, and password. + * + *

A plain text may be used for tests to a local database. Replace Cloud SQL instance name with + * JDBC URL. + */ + private static final String SQL_CONN_INFO_FILE_PATTERN = + "gs://domain-registry-dev-deploy/cloudsql-credentials/%s/admin_credential.enc"; + + private static final ImmutableSet ALLOWED_ENV = + ImmutableSet.of("alpha", "crash", "sandbox", "production"); + /** * Returns a regex pattern that matches all Datastore export files of a given {@code kind}. * @@ -90,4 +108,10 @@ public final class BackupPaths { checkArgument(start >= 0, "Illegal file name %s.", fileName); return DateTime.parse(fileName.substring(start + COMMIT_LOG_NAME_PREFIX.length())); } + + public static ImmutableList getCloudSQLCredentialFilePatterns(String environmentName) { + checkArgument( + ALLOWED_ENV.contains(environmentName), "Invalid environment name %s", environmentName); + return ImmutableList.of(String.format(SQL_CONN_INFO_FILE_PATTERN, environmentName)); + } } diff --git a/core/src/main/java/google/registry/beam/initsql/BeamJpaModule.java b/core/src/main/java/google/registry/beam/initsql/BeamJpaModule.java new file mode 100644 index 000000000..520cb4917 --- /dev/null +++ b/core/src/main/java/google/registry/beam/initsql/BeamJpaModule.java @@ -0,0 +1,191 @@ +// Copyright 2020 The Nomulus Authors. All Rights Reserved. +// +// 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 google.registry.beam.initsql; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.isNullOrEmpty; + +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; +import dagger.Binds; +import dagger.Component; +import dagger.Lazy; +import dagger.Module; +import dagger.Provides; +import google.registry.beam.initsql.BeamJpaModule.BindModule; +import google.registry.config.CredentialModule; +import google.registry.config.RegistryConfig.Config; +import google.registry.keyring.kms.KmsModule; +import google.registry.persistence.PersistenceModule; +import google.registry.persistence.PersistenceModule.JdbcJpaTm; +import google.registry.persistence.PersistenceModule.SocketFactoryJpaTm; +import google.registry.persistence.transaction.JpaTransactionManager; +import google.registry.util.Clock; +import google.registry.util.Sleeper; +import google.registry.util.SystemClock; +import google.registry.util.SystemSleeper; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.List; +import javax.inject.Named; +import javax.inject.Singleton; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResourceId; + +/** + * Provides bindings for {@link JpaTransactionManager} to Cloud SQL. + * + *

This module is intended for use in BEAM pipelines, and uses a BEAM utility to access GCS like + * a regular file system. + * + *

Note that {@link google.registry.config.RegistryConfig.ConfigModule} cannot be used here, + * since many bindings, especially KMS-related ones, are different. + */ +@Module(includes = {BindModule.class}) +class BeamJpaModule { + + private static final String GCS_SCHEME = "gs://"; + + private final String credentialFilePath; + + /** + * Constructs a new instance of {@link BeamJpaModule}. + * + * @param credentialFilePath the path to a Cloud SQL credential file. This must refer to either a + * real encrypted file on GCS as returned by {@link + * BackupPaths#getCloudSQLCredentialFilePatterns} or an unencrypted file on local filesystem + * with credentials to a test database. + */ + BeamJpaModule(String credentialFilePath) { + checkArgument(!isNullOrEmpty(credentialFilePath), "Null or empty credentialFilePath"); + this.credentialFilePath = credentialFilePath; + } + + /** Returns true if the credential file is on GCS (and therefore expected to be encrypted). */ + private boolean isCloudSqlCredential() { + return credentialFilePath.startsWith(GCS_SCHEME); + } + + @Provides + @Singleton + SqlAccessInfo provideCloudSqlAccessInfo(Lazy lazyDecryptor) { + String line = readOnlyLineFromCredentialFile(); + if (isCloudSqlCredential()) { + line = lazyDecryptor.get().decrypt(line); + } + // See ./BackupPaths.java for explanation of the line format. + List parts = Splitter.on(' ').splitToList(line.trim()); + checkState(parts.size() == 3, "Expecting three phrases in %s", line); + if (isCloudSqlCredential()) { + return SqlAccessInfo.createCloudSqlAccessInfo(parts.get(0), parts.get(1), parts.get(2)); + } else { + return SqlAccessInfo.createLocalSqlAccessInfo(parts.get(0), parts.get(1), parts.get(2)); + } + } + + String readOnlyLineFromCredentialFile() { + try { + ResourceId resourceId = FileSystems.matchSingleFileSpec(credentialFilePath).resourceId(); + try (BufferedReader reader = + new BufferedReader( + new InputStreamReader( + Channels.newInputStream(FileSystems.open(resourceId)), StandardCharsets.UTF_8))) { + return reader.readLine(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Provides + @Config("cloudSqlJdbcUrl") + String provideJdbcUrl(SqlAccessInfo sqlAccessInfo) { + return sqlAccessInfo.jdbcUrl(); + } + + @Provides + @Config("cloudSqlInstanceConnectionName") + String provideSqlInstanceName(SqlAccessInfo sqlAccessInfo) { + return sqlAccessInfo + .cloudSqlInstanceName() + .orElseThrow(() -> new IllegalStateException("Cloud SQL not provisioned.")); + } + + @Provides + @Config("cloudSqlUsername") + String provideSqlUsername(SqlAccessInfo sqlAccessInfo) { + return sqlAccessInfo.user(); + } + + @Provides + @Config("cloudSqlPassword") + String provideSqlPassword(SqlAccessInfo sqlAccessInfo) { + return sqlAccessInfo.password(); + } + + @Provides + @Config("cloudKmsProjectId") + static String kmsProjectId() { + return "domain-registry-dev"; + } + + @Provides + @Config("cloudKmsKeyRing") + static String keyRingName() { + return "nomulus-tool-keyring"; + } + + @Provides + @Config("defaultCredentialOauthScopes") + static ImmutableList defaultCredentialOauthScopes() { + return ImmutableList.of("https://www.googleapis.com/auth/cloud-platform"); + } + + @Provides + @Named("transientFailureRetries") + static int transientFailureRetries() { + return 12; + } + + @Module + interface BindModule { + + @Binds + Sleeper sleeper(SystemSleeper sleeper); + + @Binds + Clock clock(SystemClock clock); + } + + @Singleton + @Component( + modules = { + CredentialModule.class, + BeamJpaModule.class, + KmsModule.class, + PersistenceModule.class + }) + public interface JpaTransactionManagerComponent { + @SocketFactoryJpaTm + JpaTransactionManager cloudSqlJpaTransactionManager(); + + @JdbcJpaTm + JpaTransactionManager localDbJpaTransactionManager(); + } +} diff --git a/core/src/main/java/google/registry/beam/initsql/CloudSqlCredentialDecryptor.java b/core/src/main/java/google/registry/beam/initsql/CloudSqlCredentialDecryptor.java new file mode 100644 index 000000000..c93f588a7 --- /dev/null +++ b/core/src/main/java/google/registry/beam/initsql/CloudSqlCredentialDecryptor.java @@ -0,0 +1,49 @@ +// Copyright 2020 The Nomulus Authors. All Rights Reserved. +// +// 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 google.registry.beam.initsql; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.api.services.cloudkms.v1.model.DecryptRequest; +import com.google.common.base.Strings; +import google.registry.keyring.kms.KmsConnection; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import javax.inject.Inject; + +/** + * Decrypts data using Cloud KMS, with the same crypto key with which Cloud SQL credential files on + * GCS was encrypted. See {@link BackupPaths#getCloudSQLCredentialFilePatterns} for more + * information. + */ +public class CloudSqlCredentialDecryptor { + + private static final String CRYPTO_KEY_NAME = "nomulus-tool-key"; + private final KmsConnection kmsConnection; + + @Inject + CloudSqlCredentialDecryptor(KmsConnection kmsConnection) { + this.kmsConnection = kmsConnection; + } + + public String decrypt(String data) { + checkArgument(!Strings.isNullOrEmpty(data), "Null or empty data."); + byte[] ciphertext = Base64.getDecoder().decode(data); + // Re-encode for Cloud KMS JSON REST API, invoked through kmsConnection. + String urlSafeCipherText = new DecryptRequest().encodeCiphertext(ciphertext).getCiphertext(); + return new String( + kmsConnection.decrypt(CRYPTO_KEY_NAME, urlSafeCipherText), StandardCharsets.UTF_8); + } +} diff --git a/core/src/main/java/google/registry/beam/initsql/SqlAccessInfo.java b/core/src/main/java/google/registry/beam/initsql/SqlAccessInfo.java new file mode 100644 index 000000000..7b055666d --- /dev/null +++ b/core/src/main/java/google/registry/beam/initsql/SqlAccessInfo.java @@ -0,0 +1,45 @@ +// Copyright 2020 The Nomulus Authors. All Rights Reserved. +// +// 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 google.registry.beam.initsql; + +import com.google.auto.value.AutoValue; +import java.util.Optional; + +/** + * Information needed to connect to a database, including JDBC URL, user name, password, and in the + * case of Cloud SQL, the database instance's name. + */ +@AutoValue +abstract class SqlAccessInfo { + + abstract String jdbcUrl(); + + abstract String user(); + + abstract String password(); + + abstract Optional cloudSqlInstanceName(); + + public static SqlAccessInfo createCloudSqlAccessInfo( + String sqlInstanceName, String username, String password) { + return new AutoValue_SqlAccessInfo( + "jdbc:postgresql://google/postgres", username, password, Optional.of(sqlInstanceName)); + } + + public static SqlAccessInfo createLocalSqlAccessInfo( + String jdbcUrl, String username, String password) { + return new AutoValue_SqlAccessInfo(jdbcUrl, username, password, Optional.empty()); + } +} diff --git a/core/src/main/java/google/registry/beam/initsql/Transforms.java b/core/src/main/java/google/registry/beam/initsql/Transforms.java index 343e5aa29..3f05997dc 100644 --- a/core/src/main/java/google/registry/beam/initsql/Transforms.java +++ b/core/src/main/java/google/registry/beam/initsql/Transforms.java @@ -75,6 +75,11 @@ public final class Transforms { return toStringPCollection(getExportFilePatterns(exportDir, kinds)); } + public static PTransform> getCloudSqlConnectionInfoFilePatterns( + String gcpProjectName) { + return toStringPCollection(BackupPaths.getCloudSQLCredentialFilePatterns(gcpProjectName)); + } + /** * Returns a {@link PTransform} from file name patterns to file {@link Metadata Metadata records}. */ diff --git a/core/src/main/java/google/registry/keyring/kms/KmsConnection.java b/core/src/main/java/google/registry/keyring/kms/KmsConnection.java index 593de8493..5d4486746 100644 --- a/core/src/main/java/google/registry/keyring/kms/KmsConnection.java +++ b/core/src/main/java/google/registry/keyring/kms/KmsConnection.java @@ -17,7 +17,7 @@ package google.registry.keyring.kms; import google.registry.keyring.api.KeyringException; /** An abstraction to simplify Cloud KMS operations. */ -interface KmsConnection { +public interface KmsConnection { /** * The maximum allowable secret size, as set by Cloud KMS. diff --git a/core/src/main/java/google/registry/persistence/PersistenceModule.java b/core/src/main/java/google/registry/persistence/PersistenceModule.java index 511462cc3..cdef145bd 100644 --- a/core/src/main/java/google/registry/persistence/PersistenceModule.java +++ b/core/src/main/java/google/registry/persistence/PersistenceModule.java @@ -58,9 +58,10 @@ public class PersistenceModule { public static final String HIKARI_DS_CLOUD_SQL_INSTANCE = "hibernate.hikari.dataSource.cloudSqlInstance"; + @VisibleForTesting @Provides @DefaultHibernateConfigs - public static ImmutableMap providesDefaultDatabaseConfigs() { + public static ImmutableMap provideDefaultDatabaseConfigs() { ImmutableMap.Builder properties = ImmutableMap.builder(); properties.put(Environment.DRIVER, "org.postgresql.Driver"); @@ -89,7 +90,7 @@ public class PersistenceModule { @Provides @Singleton @PartialCloudSqlConfigs - public static ImmutableMap providesPartialCloudSqlConfigs( + static ImmutableMap providePartialCloudSqlConfigs( @Config("cloudSqlJdbcUrl") String jdbcUrl, @Config("cloudSqlInstanceConnectionName") String instanceConnectionName, @DefaultHibernateConfigs ImmutableMap defaultConfigs) { @@ -103,7 +104,7 @@ public class PersistenceModule { @Provides @Singleton @AppEngineJpaTm - public static JpaTransactionManager providesAppEngineJpaTm( + static JpaTransactionManager provideAppEngineJpaTm( @Config("cloudSqlUsername") String username, KmsKeyring kmsKeyring, @PartialCloudSqlConfigs ImmutableMap cloudSqlConfigs, @@ -117,7 +118,7 @@ public class PersistenceModule { @Provides @Singleton @NomulusToolJpaTm - public static JpaTransactionManager providesNomulusToolJpaTm( + static JpaTransactionManager provideNomulusToolJpaTm( @Config("toolsCloudSqlUsername") String username, KmsKeyring kmsKeyring, @PartialCloudSqlConfigs ImmutableMap cloudSqlConfigs, @@ -130,9 +131,39 @@ public class PersistenceModule { return new JpaTransactionManagerImpl(create(overrides), clock); } + @Provides + @Singleton + @SocketFactoryJpaTm + static JpaTransactionManager provideSocketFactoryJpaTm( + @Config("cloudSqlUsername") String username, + @Config("cloudSqlPassword") String password, + @PartialCloudSqlConfigs ImmutableMap cloudSqlConfigs, + Clock clock) { + HashMap overrides = Maps.newHashMap(cloudSqlConfigs); + overrides.put(Environment.USER, username); + overrides.put(Environment.PASS, password); + return new JpaTransactionManagerImpl(create(overrides), clock); + } + + @Provides + @Singleton + @JdbcJpaTm + static JpaTransactionManager provideLocalJpaTm( + @Config("cloudSqlJdbcUrl") String jdbcUrl, + @Config("cloudSqlUsername") String username, + @Config("cloudSqlPassword") String password, + @DefaultHibernateConfigs ImmutableMap defaultConfigs, + Clock clock) { + HashMap overrides = Maps.newHashMap(defaultConfigs); + overrides.put(Environment.URL, jdbcUrl); + overrides.put(Environment.USER, username); + overrides.put(Environment.PASS, password); + return new JpaTransactionManagerImpl(create(overrides), clock); + } + /** Constructs the {@link EntityManagerFactory} instance. */ @VisibleForTesting - public static EntityManagerFactory create( + static EntityManagerFactory create( String jdbcUrl, String username, String password, ImmutableMap configs) { HashMap properties = Maps.newHashMap(configs); properties.put(Environment.URL, jdbcUrl); @@ -165,6 +196,23 @@ public class PersistenceModule { @Documented public @interface NomulusToolJpaTm {} + /** + * Dagger qualifier for {@link JpaTransactionManager} that accesses Cloud SQL using socket + * factory. This is meant for applications not running on AppEngine, therefore without access to a + * {@link google.registry.keyring.api.Keyring}. + */ + @Qualifier + @Documented + public @interface SocketFactoryJpaTm {} + + /** + * Dagger qualifier for {@link JpaTransactionManager} backed by plain JDBC connections. This is + * mainly used by tests. + */ + @Qualifier + @Documented + public @interface JdbcJpaTm {} + /** Dagger qualifier for the partial Cloud SQL configs. */ @Qualifier @Documented diff --git a/core/src/test/java/google/registry/beam/initsql/BackupPathsTest.java b/core/src/test/java/google/registry/beam/initsql/BackupPathsTest.java new file mode 100644 index 000000000..349944fe6 --- /dev/null +++ b/core/src/test/java/google/registry/beam/initsql/BackupPathsTest.java @@ -0,0 +1,63 @@ +// Copyright 2020 The Nomulus Authors. All Rights Reserved. +// +// 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 google.registry.beam.initsql; + +import static com.google.common.truth.Truth.assertThat; +import static google.registry.beam.initsql.BackupPaths.getCloudSQLCredentialFilePatterns; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import org.junit.jupiter.api.Test; + +/** Unit tests for {@link google.registry.beam.initsql.BackupPaths}. */ +public class BackupPathsTest { + + @Test + void getCloudSQLCredentialFilePatterns_alpha() { + assertThat(getCloudSQLCredentialFilePatterns("alpha")) + .containsExactly( + "gs://domain-registry-dev-deploy/cloudsql-credentials/alpha/admin_credential.enc"); + } + + @Test + void getCloudSQLCredentialFilePatterns_crash() { + assertThat(getCloudSQLCredentialFilePatterns("crash")) + .containsExactly( + "gs://domain-registry-dev-deploy/cloudsql-credentials/crash/admin_credential.enc"); + } + + @Test + void getCloudSQLCredentialFilePatterns_sandbox() { + assertThat(getCloudSQLCredentialFilePatterns("sandbox")) + .containsExactly( + "gs://domain-registry-dev-deploy/cloudsql-credentials/sandbox/admin_credential.enc"); + } + + @Test + void getCloudSQLCredentialFilePatterns_production() { + assertThat(getCloudSQLCredentialFilePatterns("production")) + .containsExactly( + "gs://domain-registry-dev-deploy/cloudsql-credentials/production/admin_credential.enc"); + } + + @Test + void getEnvFromProject_illegal() { + assertThrows(IllegalArgumentException.class, () -> getCloudSQLCredentialFilePatterns("bad")); + } + + @Test + void getEnvFromProject_null() { + assertThrows(IllegalArgumentException.class, () -> getCloudSQLCredentialFilePatterns(null)); + } +} diff --git a/core/src/test/java/google/registry/beam/initsql/BeamJpaModuleTest.java b/core/src/test/java/google/registry/beam/initsql/BeamJpaModuleTest.java new file mode 100644 index 000000000..347c967f3 --- /dev/null +++ b/core/src/test/java/google/registry/beam/initsql/BeamJpaModuleTest.java @@ -0,0 +1,96 @@ +// Copyright 2020 The Nomulus Authors. All Rights Reserved. +// +// 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 google.registry.beam.initsql; + +import static com.google.common.truth.Truth.assertThat; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assume.assumeThat; + +import google.registry.persistence.NomulusPostgreSql; +import google.registry.persistence.transaction.JpaTransactionManager; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.testcontainers.containers.PostgreSQLContainer; + +/** Unit tests for {@link BeamJpaModule}. */ +@RunWith(JUnit4.class) // TODO(weiminyu): upgrade to JUnit 5. +public class BeamJpaModuleTest { + + @Rule + public PostgreSQLContainer database = new PostgreSQLContainer(NomulusPostgreSql.getDockerTag()); + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private File credentialFile; + + @Before + public void beforeEach() throws IOException { + credentialFile = temporaryFolder.newFile(); + new PrintStream(credentialFile) + .printf("%s %s %s", database.getJdbcUrl(), database.getUsername(), database.getPassword()) + .close(); + } + + @Test + public void getJpaTransactionManager_local() { + JpaTransactionManager jpa = + DaggerBeamJpaModule_JpaTransactionManagerComponent.builder() + .beamJpaModule(new BeamJpaModule(credentialFile.getAbsolutePath())) + .build() + .localDbJpaTransactionManager(); + assertThat( + jpa.transact( + () -> jpa.getEntityManager().createNativeQuery("select 1").getSingleResult())) + .isEqualTo(1); + } + + /** + * Integration test with a GCP project, only run when the 'test.gcp_integration.env' property is + * defined. Otherwise this test is ignored. This is meant to be run from a developer's desktop, + * with auth already set up by gcloud. + * + *

Example: {@code gradlew test -P test.gcp_integration.env=alpha}. + * + *

See java_common.gradle for more + * information. + */ + @Test + public void getJpaTransactionManager_cloudSql_authRequired() { + String environmentName = System.getProperty("test.gcp_integration.env"); + assumeThat(environmentName, notNullValue()); + + FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create()); + JpaTransactionManager jpa = + DaggerBeamJpaModule_JpaTransactionManagerComponent.builder() + .beamJpaModule( + new BeamJpaModule( + BackupPaths.getCloudSQLCredentialFilePatterns(environmentName).get(0))) + .build() + .cloudSqlJpaTransactionManager(); + assertThat( + jpa.transact( + () -> jpa.getEntityManager().createNativeQuery("select 1").getSingleResult())) + .isEqualTo(1); + } +} diff --git a/core/src/test/java/google/registry/persistence/PersistenceModuleTest.java b/core/src/test/java/google/registry/persistence/PersistenceModuleTest.java index f7bffea75..93c68a8e7 100644 --- a/core/src/test/java/google/registry/persistence/PersistenceModuleTest.java +++ b/core/src/test/java/google/registry/persistence/PersistenceModuleTest.java @@ -42,7 +42,7 @@ public class PersistenceModuleTest { database.getJdbcUrl(), database.getUsername(), database.getPassword(), - PersistenceModule.providesDefaultDatabaseConfigs()); + PersistenceModule.provideDefaultDatabaseConfigs()); } @AfterEach diff --git a/core/src/test/java/google/registry/persistence/transaction/JpaTransactionManagerRule.java b/core/src/test/java/google/registry/persistence/transaction/JpaTransactionManagerRule.java index e84567dc0..fc06b53ee 100644 --- a/core/src/test/java/google/registry/persistence/transaction/JpaTransactionManagerRule.java +++ b/core/src/test/java/google/registry/persistence/transaction/JpaTransactionManagerRule.java @@ -166,7 +166,7 @@ abstract class JpaTransactionManagerRule extends ExternalResource { new String(Files.readAllBytes(tempSqlFile.toPath()), StandardCharsets.UTF_8)); } - ImmutableMap properties = PersistenceModule.providesDefaultDatabaseConfigs(); + ImmutableMap properties = PersistenceModule.provideDefaultDatabaseConfigs(); if (!userProperties.isEmpty()) { // If there are user properties, create a new properties object with these added. Map mergedProperties = Maps.newHashMap(); diff --git a/java_common.gradle b/java_common.gradle index 94105056d..733a32565 100644 --- a/java_common.gradle +++ b/java_common.gradle @@ -76,6 +76,27 @@ test { useJUnitPlatform() } +// Sets up integration test with a registry environment. The target environment is +// passed by the 'test.gcp_integration.env' property. Test runner must have been +// authorized to access the corresponding GCP project, e.g., by running 'gcloud auth' +// or placing a credential file at a well known place. +// +// A typical use case is to run tests from desktop that accesses Cloud resources. See +// core/src/test/java/google/registry/beam/initsql/BeamJpaModuleTest.java for an example. +tasks.withType(Test).configureEach { + def gcp_integration_env_property = 'test.gcp_integration.env' + + if (project.hasProperty(gcp_integration_env_property)) { + String targetEnv = project.property(gcp_integration_env_property) + + if (targetEnv in ['sandbox', 'production']) { + throw new RuntimeException("Integration test with production or sandbox not allowed.") + } + systemProperty gcp_integration_env_property, targetEnv + + } +} + tasks.withType(JavaCompile).configureEach { // The -Werror flag causes Intellij to fail on deprecated api use. // Allow IDE user to turn off this flag by specifying a Gradle VM