|
| 1 | +# |
| 2 | +# Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | +# contributor license agreements. See the NOTICE file distributed with |
| 4 | +# this work for additional information regarding copyright ownership. |
| 5 | +# The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | +# (the "License"); you may not use this file except in compliance with |
| 7 | +# the License. You may obtain a copy of the License at |
| 8 | +# |
| 9 | +# http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +# |
| 11 | +# Unless required by applicable law or agreed to in writing, software |
| 12 | +# distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | +# See the License for the specific language governing permissions and |
| 15 | +# limitations under the License. |
| 16 | +# |
| 17 | + |
| 18 | +""" |
| 19 | +Demonstrates admission control in Python streaming data sources. |
| 20 | +
|
| 21 | +This example implements a simple blockchain-like streaming source that generates |
| 22 | +sequential blocks and shows how to use admission control to limit batch sizes. |
| 23 | +
|
| 24 | +Usage: structured_blockchain_admission_control.py [<max-blocks-per-batch>] |
| 25 | + <max-blocks-per-batch> Maximum number of blocks to process per microbatch (default: 10) |
| 26 | +
|
| 27 | +Run the example: |
| 28 | + `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_blockchain_admission_control.py 5` |
| 29 | +
|
| 30 | +The example will process blocks in controlled batches of 5, demonstrating admission control. |
| 31 | +""" |
| 32 | +import sys |
| 33 | +import time |
| 34 | + |
| 35 | +from pyspark.sql import SparkSession |
| 36 | +from pyspark.sql.datasource import DataSource, DataSourceStreamReader, InputPartition |
| 37 | +from pyspark.sql.types import StructType, StructField, IntegerType, LongType, StringType |
| 38 | + |
| 39 | + |
| 40 | +class SimpleBlockchainReader(DataSourceStreamReader): |
| 41 | + """A simple streaming source that generates sequential blockchain blocks.""" |
| 42 | + |
| 43 | + def __init__(self, max_block=1000): |
| 44 | + self.max_block = max_block |
| 45 | + self.current_block = 0 |
| 46 | + |
| 47 | + def initialOffset(self): |
| 48 | + """Start from block 0.""" |
| 49 | + return {"block": self.current_block} |
| 50 | + |
| 51 | + def latestOffset(self, start=None, limit=None): |
| 52 | + """ |
| 53 | + Return the latest offset, respecting admission control limits. |
| 54 | +
|
| 55 | + This demonstrates the key admission control pattern: |
| 56 | + - Without limit: process all available blocks |
| 57 | + - With maxRows limit: cap the end block to respect batch size |
| 58 | + """ |
| 59 | + # Determine where we are now |
| 60 | + if start is None: |
| 61 | + start_block = self.current_block |
| 62 | + else: |
| 63 | + start_block = start["block"] |
| 64 | + |
| 65 | + # Simulate blockchain growth - advance by 20 blocks each time |
| 66 | + latest_available = min(start_block + 20, self.max_block) |
| 67 | + |
| 68 | + # Apply admission control if configured |
| 69 | + if limit and limit.get("type") == "maxRows": |
| 70 | + max_blocks = limit["maxRows"] |
| 71 | + # Cap at the configured limit |
| 72 | + end_block = min(start_block + max_blocks, latest_available) |
| 73 | + print( |
| 74 | + f" [Admission Control] Start: {start_block}, Available: {latest_available}, " |
| 75 | + f"Capped: {end_block} (limit: {max_blocks})" |
| 76 | + ) |
| 77 | + # Return tuple: (capped_offset, true_latest_offset) |
| 78 | + return ({"block": end_block}, {"block": latest_available}) |
| 79 | + else: |
| 80 | + # No limit - process all available |
| 81 | + end_block = latest_available |
| 82 | + print(f" [No Limit] Start: {start_block}, End: {end_block}") |
| 83 | + return {"block": end_block} |
| 84 | + |
| 85 | + def partitions(self, start, end): |
| 86 | + """Create a single partition for the block range.""" |
| 87 | + start_block = start["block"] |
| 88 | + end_block = end["block"] |
| 89 | + return [InputPartition(f"{start_block}:{end_block}".encode())] |
| 90 | + |
| 91 | + def read(self, partition): |
| 92 | + """Generate block data for the partition.""" |
| 93 | + # Parse the block range |
| 94 | + range_str = partition.value.decode() |
| 95 | + start_block, end_block = map(int, range_str.split(":")) |
| 96 | + |
| 97 | + # Generate block data |
| 98 | + for block_num in range(start_block, end_block): |
| 99 | + # Simulate block data: block number, timestamp, simple hash |
| 100 | + yield ( |
| 101 | + block_num, |
| 102 | + int(time.time() * 1000), |
| 103 | + f"0x{'0' * 60}{block_num:04x}", |
| 104 | + ) |
| 105 | + |
| 106 | + def commit(self, end): |
| 107 | + """Mark this offset as committed.""" |
| 108 | + pass |
| 109 | + |
| 110 | + |
| 111 | +class SimpleBlockchainSource(DataSource): |
| 112 | + """Data source for simple blockchain streaming.""" |
| 113 | + |
| 114 | + @classmethod |
| 115 | + def name(cls): |
| 116 | + return "simple_blockchain" |
| 117 | + |
| 118 | + def schema(self): |
| 119 | + return "block_number INT, timestamp LONG, block_hash STRING" |
| 120 | + |
| 121 | + def streamReader(self, schema): |
| 122 | + return SimpleBlockchainReader(max_block=1000) |
| 123 | + |
| 124 | + |
| 125 | +if __name__ == "__main__": |
| 126 | + max_blocks_per_batch = int(sys.argv[1]) if len(sys.argv) > 1 else 10 |
| 127 | + |
| 128 | + print( |
| 129 | + f""" |
| 130 | +================================================================= |
| 131 | +Blockchain Streaming with Admission Control |
| 132 | +================================================================= |
| 133 | +Configuration: |
| 134 | + - Max blocks per batch: {max_blocks_per_batch} |
| 135 | + - Total blocks to generate: 1000 |
| 136 | +
|
| 137 | +Watch how admission control limits each microbatch to process |
| 138 | +only {max_blocks_per_batch} blocks at a time, even when more data is available. |
| 139 | +================================================================= |
| 140 | +""" |
| 141 | + ) |
| 142 | + |
| 143 | + spark = ( |
| 144 | + SparkSession.builder.appName("StructuredBlockchainAdmissionControl").getOrCreate() |
| 145 | + ) |
| 146 | + |
| 147 | + # Register the custom data source |
| 148 | + spark.dataSource.register(SimpleBlockchainSource) |
| 149 | + |
| 150 | + # Create streaming DataFrame with admission control |
| 151 | + blocks = ( |
| 152 | + spark.readStream.format("simple_blockchain") |
| 153 | + .option("maxRecordsPerBatch", str(max_blocks_per_batch)) |
| 154 | + .load() |
| 155 | + ) |
| 156 | + |
| 157 | + # Show block statistics per microbatch |
| 158 | + query = ( |
| 159 | + blocks.writeStream.outputMode("append") |
| 160 | + .format("console") |
| 161 | + .option("numRows", "20") |
| 162 | + .option("truncate", "false") |
| 163 | + .trigger(processingTime="3 seconds") |
| 164 | + .start() |
| 165 | + ) |
| 166 | + |
| 167 | + query.awaitTermination() |
| 168 | + |
0 commit comments