Skip to content

[SPARK-22732] Add Structured Streaming APIs to DataSourceV2 #19925

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 33 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
cbf7837
add tests
jose-torres Dec 5, 2017
10ac599
writer impl
jose-torres Dec 5, 2017
5594f7a
rm useless writer
jose-torres Dec 5, 2017
a364fa6
rm weird docs
jose-torres Dec 5, 2017
41d732c
shuffle around public interfaces
jose-torres Dec 5, 2017
60c12c2
fix imports
jose-torres Dec 5, 2017
57db413
put deserialize in reader so we don't have to port SerializedOffset
jose-torres Dec 5, 2017
93b6976
off by one errors grr
jose-torres Dec 5, 2017
abd20db
document right semantics
jose-torres Dec 5, 2017
3629d27
document checkpoint location
jose-torres Dec 6, 2017
d021f31
add getStart to continuous and clarify semantics
jose-torres Dec 6, 2017
052808a
cleanup offset set/get docs
jose-torres Dec 6, 2017
7a7638b
cleanup reader docs
jose-torres Dec 6, 2017
81efbee
explain getOffset
jose-torres Dec 6, 2017
a9c43d9
fix fmt
jose-torres Dec 6, 2017
3f03f50
fix doc
jose-torres Dec 6, 2017
6338043
note interfaces are temporary
jose-torres Dec 6, 2017
8949571
fix wording
jose-torres Dec 6, 2017
df9f224
lifecycle
jose-torres Dec 6, 2017
ddaee34
fix offset semantic implementation
jose-torres Dec 6, 2017
1608444
remove unneeded restriction
jose-torres Dec 7, 2017
49525b4
deserializeOffset
jose-torres Dec 7, 2017
a9fbf33
add copyright headers
jose-torres Dec 7, 2017
22d07cd
rebase against datasource package change
jose-torres Dec 7, 2017
0b68873
refer properly to sink
jose-torres Dec 8, 2017
f924a8a
widen tolerance interval in continuous rate source test
jose-torres Dec 8, 2017
4d166de
widen tolerance interval
jose-torres Dec 8, 2017
7c46b33
clarify start offset
jose-torres Dec 11, 2017
8a2a4f1
add docs
jose-torres Dec 13, 2017
8809bf9
fix problems
jose-torres Dec 13, 2017
0974ac3
even timestamps
jose-torres Dec 13, 2017
c1a6322
unify offsets
jose-torres Dec 13, 2017
3cb6cee
undo spurious rename
jose-torres Dec 13, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.kafka010.KafkaSource._
import org.apache.spark.sql.sources.v2.reader.Offset
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@ package org.apache.spark.sql.kafka010

import org.apache.kafka.common.TopicPartition

import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset}
import org.apache.spark.sql.execution.streaming.SerializedOffset
import org.apache.spark.sql.sources.v2.reader.Offset

/**
* An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import org.apache.spark.sql.ForeachWriter
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.functions.{count, window}
import org.apache.spark.sql.kafka010.KafkaSourceProvider._
import org.apache.spark.sql.sources.v2.reader.Offset
import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
import org.apache.spark.sql.streaming.util.StreamManualClock
import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2;

import java.util.Optional;

import org.apache.spark.sql.sources.v2.reader.ContinuousReader;
import org.apache.spark.sql.sources.v2.reader.DataSourceV2Reader;
import org.apache.spark.sql.types.StructType;

/**
* A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to
* provide data reading ability for continuous stream processing.
*/
public interface ContinuousReadSupport extends DataSourceV2 {
/**
* Creates a {@link ContinuousReader} to scan the data from this data source.
*
* @param schema the user provided schema, or empty() if none was provided
* @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure
* recovery. Readers for the same logical source in the same query
* will be given the same checkpointLocation.
* @param options the options for the returned data source reader, which is an immutable
* case-insensitive string-to-string map.
*/
ContinuousReader createContinuousReader(Optional<StructType> schema, String checkpointLocation, DataSourceV2Options options);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2;

import java.util.Optional;

import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.execution.streaming.BaseStreamingSink;
import org.apache.spark.sql.sources.v2.writer.ContinuousWriter;
import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer;
import org.apache.spark.sql.streaming.OutputMode;
import org.apache.spark.sql.types.StructType;

/**
* A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to
* provide data writing ability for continuous stream processing.
*/
@InterfaceStability.Evolving
public interface ContinuousWriteSupport extends BaseStreamingSink {

/**
* Creates an optional {@link ContinuousWriter} to save the data to this data source. Data
* sources can return None if there is no writing needed to be done.
*
* @param queryId A unique string for the writing query. It's possible that there are many writing
* queries running at the same time, and the returned {@link DataSourceV2Writer}
* can use this id to distinguish itself from others.
* @param schema the schema of the data to be written.
* @param mode the output mode which determines what successive epoch output means to this
* sink, please refer to {@link OutputMode} for more details.
* @param options the options for the returned data source writer, which is an immutable
* case-insensitive string-to-string map.
*/
Optional<ContinuousWriter> createContinuousWriter(
String queryId,
StructType schema,
OutputMode mode,
DataSourceV2Options options);
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,13 +36,21 @@ private String toLowerCase(String key) {
return key.toLowerCase(Locale.ROOT);
}

public static DataSourceV2Options empty() {
return new DataSourceV2Options(new HashMap<>());
}

public DataSourceV2Options(Map<String, String> originalMap) {
keyLowerCasedMap = new HashMap<>(originalMap.size());
for (Map.Entry<String, String> entry : originalMap.entrySet()) {
keyLowerCasedMap.put(toLowerCase(entry.getKey()), entry.getValue());
}
}

public Map<String, String> asMap() {
return new HashMap<>(keyLowerCasedMap);
}

/**
* Returns the option value to which the specified key is mapped, case-insensitively.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2;

import java.util.Optional;

import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.sources.v2.reader.MicroBatchReader;
import org.apache.spark.sql.types.StructType;

/**
* A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to
* provide streaming micro-batch data reading ability.
*/
@InterfaceStability.Evolving
public interface MicroBatchReadSupport extends DataSourceV2 {
/**
* Creates a {@link MicroBatchReader} to read batches of data from this data source in a
* streaming query.
*
* The execution engine will create a micro-batch reader at the start of a streaming query,
* alternate calls to setOffsetRange and createReadTasks for each batch to process, and then
* call stop() when the execution is complete. Note that a single query may have multiple
* executions due to restart or failure recovery.
*
* @param schema the user provided schema, or empty() if none was provided
* @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure
* recovery. Readers for the same logical source in the same query
* will be given the same checkpointLocation.
* @param options the options for the returned data source reader, which is an immutable
* case-insensitive string-to-string map.
*/
MicroBatchReader createMicroBatchReader(
Optional<StructType> schema,
String checkpointLocation,
DataSourceV2Options options);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2;

import java.util.Optional;

import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.execution.streaming.BaseStreamingSink;
import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer;
import org.apache.spark.sql.streaming.OutputMode;
import org.apache.spark.sql.types.StructType;

/**
* A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to
* provide data writing ability and save the data from a microbatch to the data source.
*/
@InterfaceStability.Evolving
public interface MicroBatchWriteSupport extends BaseStreamingSink {

/**
* Creates an optional {@link DataSourceV2Writer} to save the data to this data source. Data
* sources can return None if there is no writing needed to be done.
*
* @param queryId A unique string for the writing query. It's possible that there are many writing
* queries running at the same time, and the returned {@link DataSourceV2Writer}
* can use this id to distinguish itself from others.
* @param epochId The uniquenumeric ID of the batch within this writing query. This is an
Copy link
Contributor

@cloud-fan cloud-fan Dec 14, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo: unique numeric

* incrementing counter representing a consistent set of data; the same batch may
* be started multiple times in failure recovery scenarios, but it will always
* contain the same records.
* @param schema the schema of the data to be written.
* @param mode the output mode which determines what successive batch output means to this
* sink, please refer to {@link OutputMode} for more details.
* @param options the options for the returned data source writer, which is an immutable
* case-insensitive string-to-string map.
*/
Optional<DataSourceV2Writer> createMicroBatchWriter(
String queryId,
long epochId,
StructType schema,
OutputMode mode,
DataSourceV2Options options);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2.reader;

import org.apache.spark.sql.sources.v2.reader.PartitionOffset;

import java.io.IOException;

/**
* A variation on {@link DataReader} for use with streaming in continuous processing mode.
*/
public interface ContinuousDataReader<T> extends DataReader<T> {
/**
* Get the offset of the current record, or the start offset if no records have been read.
*
* The execution engine will call this method along with get() to keep track of the current
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to use a real java doc link, e.g. {@link DataReader#get}

* offset. When an epoch ends, the offset of the previous record in each partition will be saved
* as a restart checkpoint.
*/
PartitionOffset getOffset();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.spark.sql.sources.v2.reader;

import org.apache.spark.sql.sources.v2.reader.PartitionOffset;
import org.apache.spark.sql.execution.streaming.BaseStreamingSource;

import java.util.Optional;

/**
* A mix-in interface for {@link DataSourceV2Reader}. Data source readers can implement this
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not a mix-in interface but a variation on DataSourceV2Reader, right?

* interface to allow reading in a continuous processing mode stream.
*
* Implementations must ensure each read task output is a {@link ContinuousDataReader}.
*/
public interface ContinuousReader extends BaseStreamingSource, DataSourceV2Reader {
/**
* Merge offsets coming from {@link ContinuousDataReader} instances in each partition to
* a single global offset.
*/
Offset mergeOffsets(PartitionOffset[] offsets);

/**
* Deserialize a JSON string into an Offset of the implementation-defined offset type.
* @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader
*/
Offset deserializeOffset(String json);

/**
* Set the desired start offset for read tasks created from this reader. The scan will start
* from the first record after the provided offset, or from an implementation-defined inferred
* starting point if no offset is provided.
*/
void setOffset(Optional<Offset> start);

/**
* Return the specified or inferred start offset for this reader.
*
* @throws IllegalStateException if setOffset has not been called
*/
Offset getStartOffset();

/**
* The execution engine will call this method in every epoch to determine if new read tasks need
* to be generated, which may be required if for example the underlying source system has had
* partitions added or removed.
*
* If true, the query will be shut down and restarted with a new reader.
*/
default boolean needsReconfiguration() {
return false;
}
}
Loading