-
Notifications
You must be signed in to change notification settings - Fork 12
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
spannerspark: initial code skeleton for read
First phase of the code skeleton that should allow .printSchema() and .show() to work correctly as well as running queries.
- Loading branch information
Showing
3 changed files
with
348 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
// Copyright 2023 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 | ||
// | ||
// 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 com.google.cloud.spark.spanner.examples; | ||
|
||
import org.apache.spark.sql.Dataset; | ||
import org.apache.spark.sql.Row; | ||
import org.apache.spark.sql.SparkSession; | ||
|
||
public class SpannerSpark { | ||
public static void main(String[] args) { | ||
SparkSession spark = SparkSession | ||
.builder() | ||
.appName("cloud spanner reading people") | ||
.getOrCreate(); | ||
|
||
|
||
Dataset<Row> df = spark.read() | ||
.format("cloud-spanner") | ||
.option("table", "people") | ||
.option("projectId", "orijtech-161805") | ||
.option("instanceId", "oragent-ws-spanner") | ||
.option("database", "oragent") | ||
.load(); | ||
df.show(); | ||
df.printSchema(); | ||
} | ||
} |
230 changes: 230 additions & 0 deletions
230
spark-3.1-spanner-lib/src/main/java/com/google/cloud/spark/spanner/SpannerSpark.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,230 @@ | ||
// Copyright 2023 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 | ||
// | ||
// 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 com.google.cloud.spark.spanner; | ||
|
||
import com.google.cloud.spanner.BatchClient; | ||
import com.google.cloud.spanner.DatabaseClient; | ||
import com.google.cloud.spanner.connection.ConnectionOptions; | ||
import com.google.cloud.spanner.BatchReadOnlyTransaction; | ||
import com.google.cloud.spanner.ReadOnlyTransaction; | ||
import com.google.cloud.spanner.Options; | ||
import com.google.cloud.spanner.ResultSet; | ||
import com.google.cloud.spanner.Spanner; | ||
import com.google.cloud.spanner.Partition; | ||
import com.google.cloud.spanner.PartitionOptions; | ||
import com.google.cloud.spanner.Statement; | ||
import com.google.cloud.spanner.TimestampBound; | ||
|
||
import org.apache.commons.lang3.StringUtils; | ||
import org.apache.spark.sql.sources.DataSourceRegister; | ||
import org.apache.spark.sql.connector.catalog.TableProvider; | ||
import org.apache.spark.sql.SparkSession; | ||
import org.apache.spark.sql.Dataset; | ||
import org.apache.spark.sql.Row; | ||
import org.apache.spark.sql.RowFactory; | ||
import org.apache.spark.sql.types.DataType; | ||
import org.apache.spark.sql.types.DataTypes; | ||
import org.apache.spark.sql.types.StructType; | ||
import org.apache.spark.sql.Encoder; | ||
|
||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.ConcurrentHashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
public class SpannerSpark implements DataSourceRegister, TableProvider { | ||
public Dataset<Row> execute(SparkSession spark, String sqlStmt) { | ||
// 1. TODO: Verify that sqlStmt is ONLY Data-Query-Language and not DML nor DDL. | ||
|
||
// 2. Spin up the parallelizing executor. | ||
ExecutorService executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); | ||
|
||
try (final BatchReadOnlyTransaction txn = | ||
this.batchClient.batchReadOnlyTransaction(TimestampBound.strong())) { | ||
List<Partition> partitions = | ||
txn.partitionQuery( | ||
PartitionOptions.getDefaultInstance(), | ||
Statement.of(sqlStmt), | ||
Options.dataBoostEnabled(true)); | ||
|
||
ConcurrentHashMap hm = new ConcurrentHashMap(partitions.size()); | ||
for (final Partition partition : partitions) { | ||
executor.execute( | ||
() -> { | ||
// Run partitions in parallel, then combine them. | ||
try (ResultSet res = txn.execute(partition)) { | ||
// Create rows per | ||
List<Row> rows = ResultSetToSparkRow(res); | ||
hm.put(partition, rows); | ||
} | ||
}); | ||
} | ||
return dataSetFromHashMap(spark, hm); | ||
} finally { | ||
executor.shutdown(); | ||
executor.awaitTermination(10, TimeUnit.MINUTES); | ||
} | ||
} | ||
|
||
private Dataset<Row> datasetFromHashMap(SparkSession spark, Map<Partition, List<Row>> hm) { | ||
List<Row> coalescedRows = new ArrayList<Row>(); | ||
hm.values().forEach(coalescedRows::addAll); | ||
Encoder<Row> rowEncoder = Encoders.bean(Row.class); | ||
return spark.createDataset(coalescedRows, rowEncoder); | ||
} | ||
|
||
private List<Row> ResultSetToSparkRow(ResultSet rs) throws Exception { | ||
List<Row> rows = new ArrayList(); | ||
while (rs.next()) { | ||
rows.add(ResultSetIndexToRow(rs)); | ||
} | ||
return rows; | ||
} | ||
|
||
private Row ResultSetIndexToRow(ResultSet rs) { | ||
Struct spannerRow = rs.getCurrentRowAsStruct(); | ||
List<Field> fields = rs.getMetadata().getRowType().getFields(); | ||
List<Object> objects = new ArrayList(); | ||
|
||
for (int i = 0; i < fields.size(); i++) { | ||
Field field = fields.get(i); | ||
String fieldTypeName = field.getType().getName(); | ||
int openBracIndex = StringUtils(spannerStrType, '('); | ||
if (openBracIndex >= 0) { | ||
fieldTypeName = StringUtils.truncate(fieldTypeName, openBracIndex); | ||
} | ||
|
||
switch (fieldTypeName) { | ||
case "BOOL": | ||
objects.add(spannerRow.getBoolean(i)); | ||
case "BYTES": | ||
return DataTypes.BinaryType; | ||
case "DATE": | ||
objects.add(spannerRow.getDate(i)); | ||
case "FLOAT64": | ||
objects.add(spannerRow.getBigDecimal(i)); | ||
case "INT64": | ||
objects.add(spannerRow.getLong(i)); | ||
case "JSON": | ||
return DataTypes.BinaryType; | ||
case "NUMERIC": | ||
objects.add(spannerRow.getBigDecimal(i)); | ||
case "STRING": | ||
objects.add(spannerRow.getString(i)); | ||
case "TIMESTAMP": | ||
objects.add(spannerRow.getTimestamp(i)); | ||
default: // "ARRAY", "STRUCT" | ||
throw new Exception("unhandled type: " + fieldTypeName); | ||
} | ||
} | ||
|
||
return RowFactory.create(objects.toArray(new Object[0])); | ||
} | ||
|
||
@Override | ||
public String shortName() { | ||
return "cloud-spanner"; | ||
} | ||
|
||
@Override | ||
public DataType getCatalystType(int sqlType, String typename, int size, MetadataBuilder mb) { | ||
switch (typename) { | ||
case "ARRAY": | ||
return DataTypes.ArrayType; | ||
case "BOOL": | ||
return DataTypes.BooleanType; | ||
case "BYTES": | ||
return DataTypes.BinaryType; | ||
case "DATE": | ||
return DataTypes.DateType; | ||
case "FLOAT64": | ||
return DataTypes.FloatType; | ||
case "INT64": | ||
return DataTypes.LongType; | ||
case "JSON": | ||
return DataTypes.BinaryType; | ||
case "NUMERIC": | ||
return DataTypes.DoubleType; // TODO: Discuss this more | ||
case "STRING": | ||
return DataTypes.StringType; | ||
case "STRUCT": | ||
return DataTypes.StructType; | ||
case "TIMESTAMP": | ||
return DataTypes.TimestampType; | ||
default: | ||
return DataTypes.NullType; | ||
} | ||
} | ||
|
||
@Override | ||
public Transform[] inferPartitioning(CaseInsensitiveStringMap options) { | ||
return null; | ||
} | ||
|
||
@Override | ||
public StructType inferSchema(CaseInsensitiveStringMap options) { | ||
return null; | ||
} | ||
|
||
@Override | ||
public boolean supportsExternalMetadata() { | ||
return false; | ||
} | ||
|
||
@Override | ||
public Table getTable(StructType schema, Transform[] partitioning, java.util.Map<String, String> properties) { | ||
// 1. Create the DatabaseClient with the provided options: | ||
// "instanceId": <INSTANCE_ID> | ||
// "projectId": <PROJECT_ID> | ||
// "databaseId": <DATABASE_ID> | ||
String spannerUri = String.format( | ||
"cloudspanner:/projects/%s/instances/%s/databases/%s", | ||
properties.get("projectId"), | ||
properties.get("instanceId"), | ||
properties.get("databaseId")); | ||
|
||
ConnectionOptions.Builder builder = ConnectionOptions.newBuilder().setUri(spannerUri); | ||
String gcpCredsUrl = properties.get("credentials"); | ||
if (gcpCredsUrl != null) { | ||
builder = builder.setCredentialsUrl(gcpCredsUrl); | ||
} | ||
ConnectionOptions opts = builder.build(); | ||
|
||
try (Spanner spanner = opts.getService()) { | ||
// 2. Acquire the Cloud Spanner client. | ||
DatabaseClient dbClient = spanner.getDatabaseClient(opts.getDatabaseId()); | ||
|
||
String tableName = properties.get("table"); | ||
// 3. Run an information schema query to get the type definition of the table. | ||
Statement stmt = | ||
Statement.newBuilder( | ||
"SELECT COLUMN_NAME, ORDINAL_POSITION, IS_NULLABLE, SPANNER_TYPE " + | ||
"FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME=@tableName " + | ||
"ORDER BY ORDINAL_POSITION") | ||
.bind("tableName").to(tableName).build(); | ||
try (final ReadOnlyTransaction txn = dbClient.singleUse()) { | ||
try (final ResultSet rs = txn.executeQuery(stmt)) { | ||
return new SpannerTable(tableName, rs); | ||
} | ||
} | ||
} | ||
|
||
return null; | ||
} | ||
} |
79 changes: 79 additions & 0 deletions
79
spark-3.1-spanner-lib/src/main/java/com/google/cloud/spark/spanner/SpannerTable.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,79 @@ | ||
// Copyright 2023 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 | ||
// | ||
// 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 com.google.cloud.spark.spanner; | ||
|
||
import com.google.cloud.spanner.Resultset; | ||
|
||
import org.apache.commons.lang3.StringUtils; | ||
import org.apache.spark.sql.types.DataType; | ||
import org.apache.spark.sql.types.DataTypes; | ||
import org.apache.spark.sql.connector.catalog.Table; | ||
|
||
public class SpannerTable implements Table { | ||
private String tableName; | ||
private StructType tableSchema; | ||
|
||
public SpannerTable(String tableName, ResultSet rs) { | ||
this.tableName = tableName; | ||
|
||
int columnSize = rs.getMetadata().getRowType().getFields().size(); | ||
// Expecting resultset columns in the ordering: | ||
// COLUMN_NAME, ORDINAL_POSITION, IS_NULLABLE, SPANNER_TYPE | ||
StructType schema = new StructType(); | ||
while (rs.next()) { | ||
Struct row = rs.getCurrentRowAsStruct(); | ||
|
||
for (int columnIndex = 0; columnIndex < columnSize; i++) { | ||
Struct column = row.getStruct(columnIndex); | ||
|
||
String columnName = column.getString(0); | ||
// Integer ordinalPosition = column.getInt(1); | ||
boolean isNullable = column.getInt(2); | ||
DataType catalogType = SpannerTable.ofSpannerStrType(column.getString(3)); | ||
schema = schema.add(columnName, catalogType, isNullable); | ||
} | ||
} | ||
return schema; | ||
} | ||
|
||
public static DataType ofSpannerStrType(String spannerStrType) { | ||
switch (spannerStrType) { | ||
case "BOOL": | ||
return DataTypes.BooleanType; | ||
case "BYTES": | ||
return DataTypes.BinaryType; | ||
case "DATE": | ||
return DataTypes.DateType; | ||
case "FLOAT64": | ||
return DataTypes.DoubleType; | ||
case "INT64": | ||
return DataTypes.LongType; | ||
case "JSON": | ||
return DataTypes.BinaryType; | ||
case "NUMERIC": | ||
return DataTypes.DoubleType; | ||
case "STRING": | ||
return DataTypes.StringType; | ||
case "TIMESTAMP": | ||
return DataTypes.TimestampType; | ||
default: // "ARRAY", "STRUCT" | ||
int openBracIndex = StringUtils(spannerStrType, '('); | ||
if (openBracIndex >= 0) { | ||
return SpannerTable.ofSpannerStrType(StringUtils.truncate(spannerStrType, openBracIndex)); | ||
} | ||
throw new Exception("unhandled type: " + fieldTypeName); | ||
} | ||
} | ||
} |