-
Notifications
You must be signed in to change notification settings - Fork 203
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
46b4c47
commit cbcd5ae
Showing
12 changed files
with
249 additions
and
301 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,24 @@ | ||
name: Duplicate Code Detection | ||
|
||
on: | ||
push: | ||
branches: [ master ] | ||
pull_request: | ||
# The branches below must be a subset of the branches above | ||
branches: [ master ] | ||
|
||
jobs: | ||
build: | ||
|
||
runs-on: ubuntu-latest | ||
|
||
steps: | ||
- uses: actions/checkout@v3 | ||
- uses: actions/setup-java@v3 | ||
with: | ||
distribution: 'temurin' | ||
java-version: '8' | ||
cache: 'maven' | ||
|
||
- name: Running Duplicate Code Detection | ||
run: ./mvnw pmd:cpd-check -Pall -Daggregate=true |
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
135 changes: 135 additions & 0 deletions
135
...ector-common/src/main/java/com/google/cloud/spark/bigquery/direct/BigQueryRDDContext.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,135 @@ | ||
/* | ||
* Copyright 2022 Google Inc. 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 com.google.cloud.spark.bigquery.direct; | ||
|
||
import com.google.cloud.bigquery.Schema; | ||
import com.google.cloud.bigquery.connector.common.BigQueryClientFactory; | ||
import com.google.cloud.bigquery.connector.common.BigQueryStorageReadRowsTracer; | ||
import com.google.cloud.bigquery.connector.common.BigQueryTracerFactory; | ||
import com.google.cloud.bigquery.connector.common.BigQueryUtil; | ||
import com.google.cloud.bigquery.connector.common.ReadRowsHelper; | ||
import com.google.cloud.bigquery.storage.v1.DataFormat; | ||
import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; | ||
import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; | ||
import com.google.cloud.bigquery.storage.v1.ReadSession; | ||
import com.google.cloud.spark.bigquery.InternalRowIterator; | ||
import com.google.cloud.spark.bigquery.ReadRowsResponseToInternalRowIteratorConverter; | ||
import com.google.cloud.spark.bigquery.SchemaConverters; | ||
import com.google.cloud.spark.bigquery.SchemaConvertersConfiguration; | ||
import com.google.cloud.spark.bigquery.SparkBigQueryConfig; | ||
import com.google.cloud.spark.bigquery.metrics.SparkMetricsSource; | ||
import com.google.common.base.Joiner; | ||
import java.io.Serializable; | ||
import java.util.Arrays; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.Optional; | ||
import org.apache.spark.InterruptibleIterator; | ||
import org.apache.spark.Partition; | ||
import org.apache.spark.SparkEnv; | ||
import org.apache.spark.TaskContext; | ||
import org.apache.spark.sql.catalyst.InternalRow; | ||
import org.apache.spark.sql.types.StructType; | ||
|
||
class BigQueryRDDContext implements Serializable { | ||
|
||
private static long serialVersionUID = -2219993393692435055L; | ||
|
||
private final Partition[] partitions; | ||
private final ReadSession readSession; | ||
private final String[] columnsInOrder; | ||
private final Schema bqSchema; | ||
private final SparkBigQueryConfig options; | ||
private final BigQueryClientFactory bigQueryClientFactory; | ||
private final BigQueryTracerFactory bigQueryTracerFactory; | ||
|
||
private List<String> streamNames; | ||
|
||
public BigQueryRDDContext( | ||
Partition[] parts, | ||
ReadSession readSession, | ||
Schema bqSchema, | ||
String[] columnsInOrder, | ||
SparkBigQueryConfig options, | ||
BigQueryClientFactory bigQueryClientFactory, | ||
BigQueryTracerFactory bigQueryTracerFactory) { | ||
|
||
this.partitions = parts; | ||
this.readSession = readSession; | ||
this.columnsInOrder = columnsInOrder; | ||
this.bigQueryClientFactory = bigQueryClientFactory; | ||
this.bigQueryTracerFactory = bigQueryTracerFactory; | ||
this.options = options; | ||
this.bqSchema = bqSchema; | ||
this.streamNames = BigQueryUtil.getStreamNames(readSession); | ||
} | ||
|
||
public scala.collection.Iterator<InternalRow> compute(Partition split, TaskContext context) { | ||
BigQueryPartition bigQueryPartition = (BigQueryPartition) split; | ||
SparkMetricsSource sparkMetricsSource = new SparkMetricsSource(); | ||
SparkEnv.get().metricsSystem().registerSource(sparkMetricsSource); | ||
BigQueryStorageReadRowsTracer tracer = | ||
bigQueryTracerFactory.newReadRowsTracer( | ||
Joiner.on(",").join(streamNames), sparkMetricsSource); | ||
|
||
ReadRowsRequest.Builder request = | ||
ReadRowsRequest.newBuilder().setReadStream(bigQueryPartition.getStream()); | ||
|
||
ReadRowsHelper readRowsHelper = | ||
new ReadRowsHelper( | ||
bigQueryClientFactory, | ||
request, | ||
options.toReadSessionCreatorConfig().toReadRowsHelperOptions(), | ||
Optional.of(tracer)); | ||
Iterator<ReadRowsResponse> readRowsResponseIterator = readRowsHelper.readRows(); | ||
|
||
StructType schema = | ||
options | ||
.getSchema() | ||
.orElse( | ||
SchemaConverters.from(SchemaConvertersConfiguration.from(options)) | ||
.toSpark(bqSchema)); | ||
|
||
ReadRowsResponseToInternalRowIteratorConverter converter; | ||
if (options.getReadDataFormat().equals(DataFormat.AVRO)) { | ||
converter = | ||
ReadRowsResponseToInternalRowIteratorConverter.avro( | ||
bqSchema, | ||
Arrays.asList(columnsInOrder), | ||
readSession.getAvroSchema().getSchema(), | ||
Optional.of(schema), | ||
Optional.of(tracer), | ||
SchemaConvertersConfiguration.from(options)); | ||
} else { | ||
converter = | ||
ReadRowsResponseToInternalRowIteratorConverter.arrow( | ||
Arrays.asList(columnsInOrder), | ||
readSession.getArrowSchema().getSerializedSchema(), | ||
Optional.of(schema), | ||
Optional.of(tracer)); | ||
} | ||
|
||
return new InterruptibleIterator<InternalRow>( | ||
context, | ||
new ScalaIterator<InternalRow>( | ||
new InternalRowIterator(readRowsResponseIterator, converter, readRowsHelper, tracer))); | ||
} | ||
|
||
public Partition[] getPartitions() { | ||
return partitions; | ||
} | ||
} |
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
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
Oops, something went wrong.