31a16fbb40
### What changes were proposed in this pull request? This PR proposes migration of [`pyspark-stubs`](https://github.com/zero323/pyspark-stubs) into Spark codebase. ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? Yes. This PR adds type annotations directly to Spark source. This can impact interaction with development tools for users, which haven't used `pyspark-stubs`. ### How was this patch tested? - [x] MyPy tests of the PySpark source ``` mypy --no-incremental --config python/mypy.ini python/pyspark ``` - [x] MyPy tests of Spark examples ``` MYPYPATH=python/ mypy --no-incremental --config python/mypy.ini examples/src/main/python/ml examples/src/main/python/sql examples/src/main/python/sql/streaming ``` - [x] Existing Flake8 linter - [x] Existing unit tests Tested against: - `mypy==0.790+dev.e959952d9001e9713d329a2f9b196705b028f894` - `mypy==0.782` Closes #29591 from zero323/SPARK-32681. Authored-by: zero323 <mszymkiewicz@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
91 lines
3.7 KiB
Python
91 lines
3.7 KiB
Python
#
|
|
# 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.
|
|
#
|
|
import time
|
|
import unittest
|
|
|
|
from pyspark import StorageLevel
|
|
from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream
|
|
from pyspark.testing.streamingutils import should_test_kinesis, kinesis_requirement_message, \
|
|
PySparkStreamingTestCase
|
|
|
|
|
|
@unittest.skipIf(not should_test_kinesis, kinesis_requirement_message)
|
|
class KinesisStreamTests(PySparkStreamingTestCase):
|
|
|
|
def test_kinesis_stream_api(self):
|
|
# Don't start the StreamingContext because we cannot test it in Jenkins
|
|
KinesisUtils.createStream(
|
|
self.ssc, "myAppNam", "mySparkStream",
|
|
"https://kinesis.us-west-2.amazonaws.com", "us-west-2",
|
|
InitialPositionInStream.LATEST, 2, StorageLevel.MEMORY_AND_DISK_2)
|
|
KinesisUtils.createStream(
|
|
self.ssc, "myAppNam", "mySparkStream",
|
|
"https://kinesis.us-west-2.amazonaws.com", "us-west-2",
|
|
InitialPositionInStream.LATEST, 2, StorageLevel.MEMORY_AND_DISK_2,
|
|
"awsAccessKey", "awsSecretKey")
|
|
|
|
def test_kinesis_stream(self):
|
|
import random
|
|
kinesisAppName = ("KinesisStreamTests-%d" % abs(random.randint(0, 10000000)))
|
|
kinesisTestUtils = self.ssc._jvm.org.apache.spark.streaming.kinesis.KinesisTestUtils(2)
|
|
try:
|
|
kinesisTestUtils.createStream()
|
|
aWSCredentials = kinesisTestUtils.getAWSCredentials()
|
|
stream = KinesisUtils.createStream(
|
|
self.ssc, kinesisAppName, kinesisTestUtils.streamName(),
|
|
kinesisTestUtils.endpointUrl(), kinesisTestUtils.regionName(),
|
|
InitialPositionInStream.LATEST, 10, StorageLevel.MEMORY_ONLY,
|
|
aWSCredentials.getAWSAccessKeyId(), aWSCredentials.getAWSSecretKey())
|
|
|
|
outputBuffer = []
|
|
|
|
def get_output(_, rdd):
|
|
for e in rdd.collect():
|
|
outputBuffer.append(e)
|
|
|
|
stream.foreachRDD(get_output)
|
|
self.ssc.start()
|
|
|
|
testData = [i for i in range(1, 11)]
|
|
expectedOutput = set([str(i) for i in testData])
|
|
start_time = time.time()
|
|
while time.time() - start_time < 120:
|
|
kinesisTestUtils.pushData(testData)
|
|
if expectedOutput == set(outputBuffer):
|
|
break
|
|
time.sleep(10)
|
|
self.assertEqual(expectedOutput, set(outputBuffer))
|
|
except:
|
|
import traceback
|
|
traceback.print_exc()
|
|
raise
|
|
finally:
|
|
self.ssc.stop(False)
|
|
kinesisTestUtils.deleteStream()
|
|
kinesisTestUtils.deleteDynamoDBTable(kinesisAppName)
|
|
|
|
|
|
if __name__ == "__main__":
|
|
from pyspark.streaming.tests.test_kinesis import * # noqa: F401
|
|
|
|
try:
|
|
import xmlrunner # type: ignore[import]
|
|
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
|
|
except ImportError:
|
|
testRunner = None
|
|
unittest.main(testRunner=testRunner, verbosity=2)
|