Skip to content
6 changes: 5 additions & 1 deletion google-cloud-spanner/clirr-ignored-differences.xml
Original file line number Diff line number Diff line change
Expand Up @@ -945,5 +945,9 @@
<method>boolean supportsExplain()</method>
</difference>


<difference>
<differenceType>7012</differenceType>
<className>com/google/cloud/spanner/DatabaseClient</className>
<method>com.google.cloud.spanner.Statement$StatementFactory getStatementFactory()</method>
</difference>
</differences>
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.google.cloud.spanner.Options.RpcPriority;
import com.google.cloud.spanner.Options.TransactionOption;
import com.google.cloud.spanner.Options.UpdateOption;
import com.google.cloud.spanner.Statement.StatementFactory;
import com.google.spanner.v1.BatchWriteResponse;
import com.google.spanner.v1.TransactionOptions.IsolationLevel;

Expand Down Expand Up @@ -606,4 +607,17 @@ ServerStream<BatchWriteResponse> batchWriteAtLeastOnce(
* idempotent, such as deleting old rows from a very large table.
*/
long executePartitionedUpdate(Statement stmt, UpdateOption... options);

/**
* Returns StatementFactory for the given dialect.
*
* <p>A {@link StatementFactory}, can be used to create statements with unnamed parameters.
*
* <p>Examples using {@link StatementFactory}
*
* <p>databaseClient.getStatementFactory().of("SELECT NAME FROM TABLE WHERE ID = ?", 10)
*/
default StatementFactory getStatementFactory() {
throw new UnsupportedOperationException("method should be overwritten");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,16 @@
import com.google.cloud.spanner.Options.UpdateOption;
import com.google.cloud.spanner.SessionPool.PooledSessionFuture;
import com.google.cloud.spanner.SpannerImpl.ClosedException;
import com.google.cloud.spanner.Statement.StatementFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.spanner.v1.BatchWriteResponse;
import io.opentelemetry.api.common.Attributes;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import javax.annotation.Nullable;

class DatabaseClientImpl implements DatabaseClient {
Expand Down Expand Up @@ -139,6 +144,30 @@ public Dialect getDialect() {
return pool.getDialect();
}

private final AbstractLazyInitializer<StatementFactory> statementFactorySupplier =
new AbstractLazyInitializer<StatementFactory>() {
@Override
protected StatementFactory initialize() {
try {
Dialect dialect = getDialectAsync().get(30, TimeUnit.SECONDS);
return new StatementFactory(dialect);
} catch (ExecutionException | TimeoutException e) {
throw SpannerExceptionFactory.asSpannerException(e);
} catch (InterruptedException e) {
throw SpannerExceptionFactory.propagateInterrupt(e);
}
}
};

@Override
public StatementFactory getStatementFactory() {
try {
return statementFactorySupplier.get();
} catch (Exception exception) {
throw SpannerExceptionFactory.asSpannerException(exception);
}
}

@Override
@Nullable
public String getDatabaseRole() {
Expand Down Expand Up @@ -346,6 +375,14 @@ public long executePartitionedUpdate(final Statement stmt, final UpdateOption...
return executePartitionedUpdateWithPooledSession(stmt, options);
}

private Future<Dialect> getDialectAsync() {
MultiplexedSessionDatabaseClient client = getMultiplexedSessionDatabaseClient();
if (client != null) {
return client.getDialectAsync();
}
return pool.getDialectAsync();
}

private long executePartitionedUpdateWithPooledSession(
final Statement stmt, final UpdateOption... options) {
ISpan span = tracer.spanBuilder(PARTITION_DML_TRANSACTION, commonAttributes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -652,6 +653,14 @@ public Dialect getDialect() {
}
}

Future<Dialect> getDialectAsync() {
try {
return MAINTAINER_SERVICE.submit(dialectSupplier::get);
} catch (Exception exception) {
throw SpannerExceptionFactory.asSpannerException(exception);
}
}

@Override
public Timestamp write(Iterable<Mutation> mutations) throws SpannerException {
return createMultiplexedSessionTransaction(/* singleUse = */ false).write(mutations);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -2548,6 +2549,10 @@ Dialect getDialect() {
}
}

Future<Dialect> getDialectAsync() {
return executor.submit(this::getDialect);
}

PooledSessionReplacementHandler getPooledSessionReplacementHandler() {
return pooledSessionReplacementHandler;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* Copyright 2025 Google LLC
*
* 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
*
* https://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 com.google.cloud.spanner;

import com.google.cloud.Date;
import com.google.protobuf.ListValue;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.OffsetDateTime;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.time.temporal.TemporalAccessor;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;

final class SpannerTypeConverter {

private static final ZoneId UTC_ZONE = ZoneId.of("UTC");
private static final DateTimeFormatter ISO_8601_DATE_FORMATTER =
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX");

static <T> Value createUntypedArrayValue(Stream<T> stream) {
List<com.google.protobuf.Value> values =
stream
.map(
val ->
com.google.protobuf.Value.newBuilder()
.setStringValue(String.valueOf(val))
.build())
.collect(Collectors.toList());
return Value.untyped(
com.google.protobuf.Value.newBuilder()
.setListValue(ListValue.newBuilder().addAllValues(values).build())
.build());
}

static <T extends TemporalAccessor> String convertToISO8601(T dateTime) {
return ISO_8601_DATE_FORMATTER.format(dateTime);
}

static <T> Value createUntypedStringValue(T value) {
return Value.untyped(
com.google.protobuf.Value.newBuilder().setStringValue(String.valueOf(value)).build());
}

static <T, U> Iterable<U> convertToTypedIterable(
Function<T, U> func, T val, Iterator<?> iterator) {
List<U> values = new ArrayList<>();
SpannerTypeConverter.processIterable(val, iterator, func, values::add);
return values;
}

static <T> Iterable<T> convertToTypedIterable(T val, Iterator<?> iterator) {
return convertToTypedIterable(v -> v, val, iterator);
}

@SuppressWarnings("unchecked")
static <T, U> void processIterable(
T val, Iterator<?> iterator, Function<T, U> func, Consumer<U> consumer) {
consumer.accept(func.apply(val));
iterator.forEachRemaining(values -> consumer.accept(func.apply((T) values)));
}

static Date convertLocalDateToSpannerDate(LocalDate date) {
return Date.fromYearMonthDay(date.getYear(), date.getMonthValue(), date.getDayOfMonth());
}

static <T> Value createUntypedIterableValue(
T value, Iterator<?> iterator, Function<T, String> func) {
ListValue.Builder listValueBuilder = ListValue.newBuilder();
SpannerTypeConverter.processIterable(
value,
iterator,
(val) -> com.google.protobuf.Value.newBuilder().setStringValue(func.apply(val)).build(),
listValueBuilder::addValues);
return Value.untyped(
com.google.protobuf.Value.newBuilder().setListValue(listValueBuilder.build()).build());
}

static ZonedDateTime atUTC(LocalDateTime localDateTime) {
return atUTC(localDateTime.atZone(ZoneId.systemDefault()));
}

static ZonedDateTime atUTC(OffsetDateTime localDateTime) {
return localDateTime.atZoneSameInstant(UTC_ZONE);
}

static ZonedDateTime atUTC(ZonedDateTime localDateTime) {
return localDateTime.withZoneSameInstant(UTC_ZONE);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import static com.google.common.base.Preconditions.checkState;

import com.google.cloud.spanner.ReadContext.QueryAnalyzeMode;
import com.google.cloud.spanner.connection.AbstractStatementParser;
import com.google.cloud.spanner.connection.AbstractStatementParser.ParametersInfo;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.spanner.v1.ExecuteSqlRequest.QueryOptions;
Expand Down Expand Up @@ -246,4 +248,90 @@ StringBuilder toString(StringBuilder b) {
}
return b;
}

/**
* Factory for creating {@link Statement}.
*
* <p>This factory class supports creating {@link Statement} with positional(or unnamed)
* parameters.
*
* <p>
*
* <h2>Usage Example</h2>
*
* Simple SQL query
*
* <pre>{@code
* Statement statement = databaseClient.getStatementFactory()
* .withUnnamedParameters("SELECT * FROM TABLE WHERE ID = ?", 10L)
* }</pre>
*
* How to use SQL queries with IN command
*
* <pre>{@code
* long[] ids = {10L, 12L, 1483L};
* Statement statement = databaseClient.getStatementFactory()
* .withUnnamedParameters("SELECT * FROM TABLE WHERE ID = UNNEST(?)", ids)
* }</pre>
*
* @see DatabaseClient#getStatementFactory()
* @see StatementFactory#withUnnamedParameters(String, Object...)
*/
public static final class StatementFactory {
private final Dialect dialect;

StatementFactory(Dialect dialect) {
this.dialect = dialect;
}

public Statement of(String sql) {
return Statement.of(sql);
}

/**
* This function accepts the SQL statement with unnamed parameters(?) and accepts the list of
* objects to replace unnamed parameters. Primitive types are supported.
*
* <p>For Date column, following types are supported
*
* <ul>
* <li>java.util.Date
* <li>LocalDate
* <li>com.google.cloud.Date
* </ul>
*
* <p>For Timestamp column, following types are supported. All the dates should be in UTC
* format. Incase if the timezone is not in UTC, spanner client will convert that to UTC
* automatically
*
* <ul>
* <li>LocalDateTime
* <li>OffsetDateTime
* <li>ZonedDateTime
* </ul>
*
* <p>
*
* @param sql SQL statement with unnamed parameters denoted as ?
* @param values positional list of values for the unnamed parameters in the SQL string
* @return Statement a statement that can be executed on Spanner
* @see DatabaseClient#getStatementFactory
*/
public Statement withUnnamedParameters(String sql, Object... values) {
Map<String, Value> parameters = getUnnamedParametersMap(values);
AbstractStatementParser statementParser = AbstractStatementParser.getInstance(this.dialect);
ParametersInfo parametersInfo =
statementParser.convertPositionalParametersToNamedParameters('?', sql);
return new Statement(parametersInfo.sqlWithNamedParameters, parameters, null);
}

private Map<String, Value> getUnnamedParametersMap(Object[] values) {
Map<String, Value> parameters = new HashMap<>();
int index = 1;
for (Object value : values) {
parameters.put("p" + (index++), Value.toValue(value));
}
return parameters;
}
}
}
Loading
Loading