├── .gitignore ├── Dockerfile ├── LICENSE ├── README.md ├── compose.yaml ├── grafana_provisioning ├── dashboards │ ├── grafana_dashboard.json │ └── main.yaml └── datasources │ └── automatic.yml ├── hackernews.py ├── init_sql └── init.sql ├── proton.py └── requirements.txt /.gitignore: -------------------------------------------------------------------------------- 1 | # Byte-compiled / optimized / DLL files 2 | __pycache__/ 3 | *.py[cod] 4 | *$py.class 5 | 6 | # C extensions 7 | *.so 8 | 9 | # Distribution / packaging 10 | .Python 11 | build/ 12 | develop-eggs/ 13 | dist/ 14 | downloads/ 15 | eggs/ 16 | .eggs/ 17 | lib/ 18 | lib64/ 19 | parts/ 20 | sdist/ 21 | var/ 22 | wheels/ 23 | share/python-wheels/ 24 | *.egg-info/ 25 | .installed.cfg 26 | *.egg 27 | MANIFEST 28 | 29 | # PyInstaller 30 | # Usually these files are written by a python script from a template 31 | # before PyInstaller builds the exe, so as to inject date/other infos into it. 32 | *.manifest 33 | *.spec 34 | 35 | # Installer logs 36 | pip-log.txt 37 | pip-delete-this-directory.txt 38 | 39 | # Unit test / coverage reports 40 | htmlcov/ 41 | .tox/ 42 | .nox/ 43 | .coverage 44 | .coverage.* 45 | .cache 46 | nosetests.xml 47 | coverage.xml 48 | *.cover 49 | *.py,cover 50 | .hypothesis/ 51 | .pytest_cache/ 52 | cover/ 53 | 54 | # Translations 55 | *.mo 56 | *.pot 57 | 58 | # Django stuff: 59 | *.log 60 | local_settings.py 61 | db.sqlite3 62 | db.sqlite3-journal 63 | 64 | # Flask stuff: 65 | instance/ 66 | .webassets-cache 67 | 68 | # Scrapy stuff: 69 | .scrapy 70 | 71 | # Sphinx documentation 72 | docs/_build/ 73 | 74 | # PyBuilder 75 | .pybuilder/ 76 | target/ 77 | 78 | # Jupyter Notebook 79 | .ipynb_checkpoints 80 | 81 | # IPython 82 | profile_default/ 83 | ipython_config.py 84 | 85 | # pyenv 86 | # For a library or package, you might want to ignore these files since the code is 87 | # intended to run in multiple environments; otherwise, check them in: 88 | # .python-version 89 | 90 | # pipenv 91 | # According to pypa/pipenv#598, it is recommended to include Pipfile.lock in version control. 92 | # However, in case of collaboration, if having platform-specific dependencies or dependencies 93 | # having no cross-platform support, pipenv may install dependencies that don't work, or not 94 | # install all needed dependencies. 95 | #Pipfile.lock 96 | 97 | # poetry 98 | # Similar to Pipfile.lock, it is generally recommended to include poetry.lock in version control. 99 | # This is especially recommended for binary packages to ensure reproducibility, and is more 100 | # commonly ignored for libraries. 101 | # https://python-poetry.org/docs/basic-usage/#commit-your-poetrylock-file-to-version-control 102 | #poetry.lock 103 | 104 | # pdm 105 | # Similar to Pipfile.lock, it is generally recommended to include pdm.lock in version control. 106 | #pdm.lock 107 | # pdm stores project-wide configurations in .pdm.toml, but it is recommended to not include it 108 | # in version control. 109 | # https://pdm.fming.dev/#use-with-ide 110 | .pdm.toml 111 | 112 | # PEP 582; used by e.g. github.com/David-OConnor/pyflow and github.com/pdm-project/pdm 113 | __pypackages__/ 114 | 115 | # Celery stuff 116 | celerybeat-schedule 117 | celerybeat.pid 118 | 119 | # SageMath parsed files 120 | *.sage.py 121 | 122 | # Environments 123 | .env 124 | .venv 125 | env/ 126 | venv/ 127 | ENV/ 128 | env.bak/ 129 | venv.bak/ 130 | 131 | # Spyder project settings 132 | .spyderproject 133 | .spyproject 134 | 135 | # Rope project settings 136 | .ropeproject 137 | 138 | # mkdocs documentation 139 | /site 140 | 141 | # mypy 142 | .mypy_cache/ 143 | .dmypy.json 144 | dmypy.json 145 | 146 | # Pyre type checker 147 | .pyre/ 148 | 149 | # pytype static type analyzer 150 | .pytype/ 151 | 152 | # Cython debug symbols 153 | cython_debug/ 154 | 155 | # PyCharm 156 | # JetBrains specific template is maintained in a separate JetBrains.gitignore that can 157 | # be found at https://github.com/github/gitignore/blob/main/Global/JetBrains.gitignore 158 | # and can be added to the global gitignore or merged into this file. For a more nuclear 159 | # option (not recommended) you can uncomment the following to ignore the entire idea folder. 160 | #.idea/ 161 | -------------------------------------------------------------------------------- /Dockerfile: -------------------------------------------------------------------------------- 1 | # syntax=docker/dockerfile:1 2 | 3 | FROM python:3.10 as base 4 | 5 | # Prevents Python from writing pyc files. 6 | ENV PYTHONDONTWRITEBYTECODE=1 7 | 8 | # Keeps Python from buffering stdout and stderr to avoid situations where 9 | # the application crashes without emitting any logs due to buffering. 10 | ENV PYTHONUNBUFFERED=1 11 | 12 | #WORKDIR /app 13 | 14 | # Create a non-privileged user that the app will run under. 15 | # See https://docs.docker.com/develop/develop-images/dockerfile_best-practices/#user 16 | ARG UID=10001 17 | RUN adduser \ 18 | --disabled-password \ 19 | --gecos "" \ 20 | --home "/nonexistent" \ 21 | --shell "/sbin/nologin" \ 22 | --no-create-home \ 23 | --uid "${UID}" \ 24 | appuser 25 | 26 | # Download dependencies as a separate step to take advantage of Docker's caching. 27 | # Leverage a cache mount to /root/.cache/pip to speed up subsequent builds. 28 | # Leverage a bind mount to requirements.txt to avoid having to copy them into 29 | # into this layer. 30 | RUN --mount=type=cache,target=/root/.cache/pip \ 31 | --mount=type=bind,source=requirements.txt,target=requirements.txt \ 32 | python -m pip install -r requirements.txt 33 | 34 | # Switch to the non-privileged user to run the application. 35 | USER appuser 36 | 37 | # Copy the source code into the container. 38 | COPY . . 39 | 40 | # Run bytewax with 5 workers to chew through the network requests on startup. 41 | CMD python -m bytewax.run hackernews.py -w 5 42 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2024 Bytewax 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ![dashboard](https://github.com/bytewax/hacking-hacker-news/assets/6073079/15c0f21a-d48d-4d11-b2ac-57f6631b147d) 2 | 3 | By @awmatheson 4 | 5 | ## Analyzing Prolific Hacker News Commenters and Trending Stories in real-time 6 | 7 | If you are like me, you eventually get tired of whatever algorithm is serving you your media. Whether it is Twitter (oops, I mean X) or Reddit, or Hacker News (HN). My recent work with the Hacker News API for a [conference talk on RAG for AI](https://www.youtube.com/watch?v=lRleB4E0Fk4&t=4s) sparked a new curiosity. After the RAG pipelines for AI work using Hacker News with Bytewax, I was curious about a new challenge: creating a customizable dashboard to serve up my HN stories in a personal way. 8 | 9 | In this blog post, I'll share how I (with support from [Jove from Timeplus](https://www.linkedin.com/in/jovezhong/)) combining different technologies ( [Bytewax](https://github.com/bytewax/bytewax), [Proton](https://github.com/timeplus-io/proton), and Grafana) to create such a personalized dashboard. The combination of the products is really fun (My opinion :slightly_smiling_face:); You get the flexibility of imperative programming with Python (Bytewax) and the power of declarative data processing through SQL (Proton). 10 | 11 | I sketched out the result below. At the end, you'll see a Grafana dashboard that dynamically shows your personalized news. It is powered by a Bytewax pipeline streaming HN stories and comments into Proton where I join and analyze the data in real time. 12 | 13 | ![hn_proton_grafana(1)](https://github.com/bytewax/hacking-hacker-news/assets/6073079/02a8658a-fb7a-493d-86c9-e3471a0b89e1) 14 | 15 | Want to skip to running the code → Run the full pipeline and dashboard with `docker compose up`. 16 | 17 | ## Introducing Bytewax 18 | 19 | Jove (One of Proton's creators) and I met for a Streaming Caffeine podcast before, I shared why I started Bytewax, and Jove talks about why they started Timeplus/Proton. Watch the whole episode [here](https://youtu.be/FJIOmB64h1k?si=ZxVkwo2SX4M_IprO)! 20 | 21 | Bytewax is an [open-source project](https://github.com/bytewax/bytewax) known for facilitating custom connections to many data sources. The often complex task of processing streaming data is much more manageable with it. For many of us, its input and output API will look friendly and familiar. The ability to run Python native code and re-use the Python libraries you already know also lowers the learning barrier. 22 | In this post, I will showcase these advantages while building a custom input connector that will read Hacker News (yes, the Hacker News) updates to keep track of trending posts and prolific commenters. 23 | 24 | ## Timeplus and Proton 25 | 26 | [Proton](https://github.com/timeplus-io/proton) is a really cool open source project built on top of the much-loved project, Clickhouse. The company behind the project, Timeplus was started by Splunk engineers out of their experience and learnings while building streaming engines at Splunk. 27 | 28 | As described on their website and marketing materials: 29 | _"Timeplus is a unified platform tailored for both streaming and historical data processing. At the heart of Timeplus is Proton, an open source project that functions as a streaming SQL engine, a fast and lightweight alternative to Apache Flink in a single binary, powered by ClickHouse. Proton is heralded for its ability to unlock streaming data value using SQL, offering a powerful end-to-end capability for processing streaming and historical data quickly and intuitively."_ 30 | 31 | _"With Proton, even on a single commodity machine, you can get 4-millisecond end-to-end latency and a benchmark of 10+ million events per second. This powerful core engine is now open-sourced under Apache 2.0 License, providing flexible deployment options with no external service dependencies."_ 32 | 33 | ## A Harmonious Partnership 34 | 35 | The unique capability to easily ingest a variety of data sources with Python in Bytewax and the power of SQL based processing of Proton is a match made in heaven! 36 | 37 | ![michie](https://github.com/bytewax/hacking-hacker-news/assets/6073079/5ab0c698-baeb-4e5e-9e69-a9bed64ecf59) 38 | 39 | 40 | ## Let’s dive into the code! 41 | 42 | ### Building a Custom Input Connector with Bytewax 43 | 44 | First things first, I need to connect to the data source. Bytewax is flexible, and I can tailor data ingestion according to my application's needs. In this case, I would like to poll data from the Hacker News API at regular time intervals. 45 | Let me walk you through a code snippet that demonstrates how to do that using Bytewax. 46 | > Imports skipped for brevity 47 | 48 | ```python 49 | class HNSource(SimplePollingSource): 50 | def next_item(self): 51 | return ( 52 | "GLOBAL_ID", 53 | requests.get("https://hacker-news.firebaseio.com/v0/maxitem.json").json(), 54 | ) 55 | ``` 56 | 57 | `_HNSource` uses the built-in [`SimplePollingSource`](https://bytewax.io/apidocs/bytewax.inputs#bytewax.inputs.SimplePollingSource) to effectively poll the HN API for the most recent ID. 58 | 59 | By following a similar pattern, you can create custom input connectors to a wide variety of APIs. See an example of a coinbase websocket custom input [here](https://github.com/bytewax/bytewax/blob/5b7a1194f28e9cf500d77420e4a7ba3ab7b8344c/examples/orderbook.py) and a wikimedia SSE stream [here](https://github.com/bytewax/bytewax/blob/5b7a1194f28e9cf500d77420e4a7ba3ab7b8344c/examples/wikistream.py). 60 | 61 | ### Writing the dataflow 62 | 63 | With our custom connector, now we can write a dataflow to describe the processing after we receive new items from the Hacker News API: 64 | 65 | #### Helper Functions: 66 | 67 | The following functions will be used inside of Bytewax operators. Operators are dataflow class methods that control the flow of data. 68 | 69 | 1. **Creating the stream with `get_id_stream`** 70 | 71 | ```python 72 | def get_id_stream(old_max_id, new_max_id) -> Tuple[str,list]: 73 | if old_max_id is None: 74 | # Get the last 150 items on the first run. 75 | old_max_id = new_max_id - 150 76 | return (new_max_id, range(old_max_id, new_max_id)) 77 | ``` 78 | 79 | 1. **download_metadata**: 80 | 81 | ```python 82 | def download_metadata(hn_id) -> Optional[Tuple[str, dict]]: 83 | # Given an hacker news id returned from the api, fetch metadata 84 | # Try 3 times, waiting more and more, or give up 85 | data = requests.get( 86 | f"https://hacker-news.firebaseio.com/v0/item/{hn_id}.json" 87 | ).json() 88 | 89 | if data is None: 90 | logger.warning(f"Couldn't fetch item {hn_id}, skipping") 91 | return None 92 | return (str(hn_id), data) 93 | ``` 94 | 95 | - This function is designed to fetch metadata for a specific Hacker News item using its ID. If the request fails to retrieve the data, it logs a warning and retries after a 0.5-second delay. 96 | 97 | 3. **recursing the comment tree**: 98 | 99 | ```python 100 | def recurse_tree(metadata, og_metadata=None) -> any: 101 | if not og_metadata: 102 | og_metadata = metadata 103 | try: 104 | parent_id = metadata["parent"] 105 | parent_metadata = download_metadata(parent_id) 106 | return recurse_tree(parent_metadata[1], og_metadata) 107 | except KeyError: 108 | return (metadata["id"], 109 | { 110 | **og_metadata, 111 | "root_id":metadata["id"] 112 | } 113 | ) 114 | 115 | 116 | def key_on_parent(key__metadata) -> tuple: 117 | key, metadata = recurse_tree(key__metadata[1]) 118 | return (str(key), metadata) 119 | ``` 120 | 121 | - In the stream, we will receive both the top-level stories and then their comments. We want to be able to attribute them properly to the top level, so we need to recurse up the comment tree to get the story data associated with the comment. 122 | 123 | 4. **Formatting our Data Correctly** 124 | 125 | ```python 126 | def format(id__metadata): 127 | id, metadata = id__metadata 128 | return json.dumps(metadata) 129 | ``` 130 | 131 | - In order to output our data to Proton, we need to pass a JSON string of the metadata dictionary so it can be parsed with Proton SQL functions. 132 | 133 | ### Defining the dataflow 134 | 135 | With the code for transformations and enrichment defined, we can move on to defining the dataflow. A dataflow is a series of steps that define the flow and transformation of data in a directed graph. 136 | 137 | 1. **Creating the Dataflow**: 138 | 139 | ```python 140 | flow = Dataflow("hn_scraper") 141 | ``` 142 | 143 | - A new `Dataflow` instance is created which acts as a blueprint for the data processing pipeline. 144 | 145 | 2. **Setting up Input**: 146 | 147 | ```python 148 | max_id = op.input("in", flow, HNSource(timedelta(seconds=15))) 149 | ``` 150 | 151 | - The `input` method is utilized to set up the data source for the dataflow. In this case, an instance of `HNInput` is created with a specified polling interval, and is tagged with the identifier "in". 152 | 153 | 3. **Data Redistribution**: 154 | 155 | ```python 156 | id_stream = op.stateful_map("range", max_id, lambda: None, get_id_stream).then( 157 | op.flat_map, "strip_key_flatten", lambda key_ids: key_ids[1]).then( 158 | op.redistribute, "redist") 159 | ``` 160 | 161 | - Building our stream and redistributing the data. In order to make our dataflow stateful and recoverable, we use a stateful our stateful function `get_id_stream` to keep track of the max_id. In that function we return a range of ids and we flat map these in order to send them downstream as a stream. 162 | - With a stream of IDs. We use the `redistribute` method to ensure that if the dataflow is run with multiple workers, the downloads in the subsequent `map` will be parallelized, enhancing the efficiency of the data processing. If you remember back to our Hacker News Input connector, we ran with only a single partition to avoid duplicate IDs. Now we need to redistribute these so we can more efficiently make a network call at the next step. 163 | 164 | 4. **Mapping to metadata function**: 165 | ```python 166 | id_stream = op.filter_map("meta_download", id_stream, download_metadata) 167 | ``` 168 | - The `filter_map` method applies the `download_metadata` function to each item in the stream. This stage is where the actual data processing (metadata downloading) occurs and if None is returned, that item is skipped and won’t be sent downstream. 169 | 170 | 5. **Splitting stories and comments**: 171 | ```python 172 | split_stream = op.branch("split_comments", id_stream, lambda item: item[1]["type"] == "story") 173 | story_stream = split_stream.trues 174 | story_stream = op.map("format_stories", story_stream, format) 175 | 176 | comment_stream = split_stream.falses 177 | comment_stream = op.map("key_on_parent", comment_stream, key_on_parent) 178 | comment_stream = op.map("format_comments", comment_stream, format) 179 | ``` 180 | - We split the stream based on the type and then we can act on the different types (comments and stories) differently as needed. 181 | - The stories are ready to be formatted in the map step to prep the data for output. 182 | - In the comment stream, we can find the story ID, which is the ultimate parent of the comment tree. We want this for our SQL analysis in Proton. Once we have the story ID we can format the comment stream for output as well. 183 | 184 | 5. **Setting up Output**: 185 | ```python 186 | op.inspect("stories", story_stream) 187 | op.inspect("comments", comment_stream) 188 | op.output("stories-out", story_stream, ProtonSink("hn_stories_raw", os.environ.get("PROTON_HOST","127.0.0.1"))) 189 | op.output("comments-out", comment_stream, ProtonSink("hn_comments_raw", os.environ.get("PROTON_HOST","127.0.0.1"))) 190 | ``` 191 | - First, we log our streams to the console/terminal, this is purely for debugging purposes and should be removed for production environments. 192 | - The `output` method is utilized to specify the output destination for the processed data. Here, a `ProtonSink` connector is utilized to output both streams to Proton. In Bytewax you can have multiple outputs in parallel with branching, you can also have multiple outputs in series if you wanted to write the same data to different locations. 193 | 194 | Let’s take a look at what writing a custom sink connector for Proton looks like. 195 | 196 | The ProtonSink will connect to a running Proton instance and create the stream if it doesn't exist. This connector uses the `raw` type, but could be modified to use a schema. The _ProtonSinkPartition is based on the Bytewax class StatelessSinkPartition which is used to define what happens on each worker at startup and for each batch of records in the stream that it receives. 197 | 198 | ```python 199 | class _ProtonSinkPartition(StatelessSinkPartition): 200 | def __init__(self, stream: str, host: str): 201 | self.client=client.Client(host=host, port=8463) 202 | self.stream=stream 203 | sql=f"CREATE STREAM IF NOT EXISTS `{stream}` (raw string)" 204 | logger.debug(sql) 205 | self.client.execute(sql) 206 | 207 | def write_batch(self, items): 208 | logger.debug(f"inserting data {items}") 209 | rows=[] 210 | for item in items: 211 | rows.append([item]) # single column in each row 212 | sql = f"INSERT INTO `{self.stream}` (raw) VALUES" 213 | logger.debug(f"inserting data {sql}") 214 | self.client.execute(sql,rows) 215 | ``` 216 | 217 | To initialize the _ProtonSinkPartition, we define a ProtonSink class based on the Dynamic Sink. This defines the behavior of Bytewax on startup and how data is split across partitions. Build is called with information about the worker and this can be used to logically divide work and IO. 218 | 219 | ``` 220 | class ProtonSink(DynamicSink): 221 | def __init__(self, stream: str, host: str): 222 | self.stream = stream 223 | self.host = host if host is not None and host != "" else "127.0.0.1" 224 | 225 | """Write each output item to Proton on that worker. 226 | 227 | Items consumed from the dataflow must look like a string. Use a 228 | proceeding map step to do custom formatting. 229 | 230 | Workers are the unit of parallelism. 231 | 232 | Can support at-least-once processing. Messages from the resume 233 | epoch will be duplicated right after resume. 234 | 235 | """ 236 | 237 | def build(self, worker_index, worker_count): 238 | """See ABC docstring.""" 239 | return _ProtonSinkPartition(self.stream, self.host) 240 | ``` 241 | 242 | With our dataflow written, move on to using proton to analyze the data. 243 | 244 | ## Real-time Hacker News Analysis 245 | 246 | Proton comes with some awesome integrations out of the box! One of those is Grafana. We will outline how you can write some queries to take the raw json strings and create some Grafana visualizations like the dashboard below. 247 | 248 | As shown in our Proton sink we are writing a batch of JSON strings as values. In our dataflow we have two streams and we will end up with two table streams in Proton. In order to use the values downstream, we need to create tables to receive the streaming data. We can do this by executing the following statements in Proton. 249 | 250 | ```SQL 251 | CREATE STREAM hn_stories_raw(raw string); 252 | CREATE STREAM hn_comments_raw(raw string); 253 | ``` 254 | 255 | Once we start to convert those raw strings into columns in materialized views with the following queries executed on Proton. These will be executed as part of the `docker compose` start if you are using it that way. 256 | 257 | ```SQL 258 | CREATE MATERIALIZED VIEW hn_stories AS 259 | SELECT 260 | to_time(raw:time) AS _tp_time, 261 | raw:id::int AS id, 262 | raw:title AS title, 263 | raw:by AS by, 264 | raw 265 | FROM hn_stories_raw; 266 | CREATE MATERIALIZED VIEW hn_comments AS 267 | SELECT 268 | to_time(raw:time) AS _tp_time, 269 | raw:id::int AS id, 270 | raw:root_id::int AS root_id, 271 | raw:by AS by, 272 | raw 273 | FROM hn_comments_raw; 274 | ``` 275 | 276 | In the SQL queries above, the `raw` command is used to parse the JSON fields. 277 | 278 | Once we have the views we can query the tables individually: 279 | 280 | ```SQL 281 | SELECT * FROM hn_stories WHERE _tp_time>earliest_ts(); 282 | ``` 283 | 284 | We can also make a view of the query above for both the stories and the comments. You can execute these directly in Proton and again this is done for you if using docker compose. 285 | 286 | ```SQL 287 | CREATE VIEW IF NOT EXISTS story AS SELECT * FROM hn_stories WHERE _tp_time>earliest_ts(); 288 | CREATE VIEW IF NOT EXISTS comment AS SELECT * FROM hn_comments WHERE _tp_time>earliest_ts() 289 | ``` 290 | 291 | With our views in hand, we can make a streaming query. You could do this in the [Grafana explore query editor](http://localhost:3000/explore): 292 | 293 | ![explore](https://github.com/bytewax/hacking-hacker-news/assets/6073079/0f0058b5-48c8-4ffe-969f-958e5cf60797) 294 | 295 | ```SQL 296 | select * from comment; 297 | 298 | select 299 | story._tp_time as story_time,comment._tp_time as comment_time, 300 | story.id as story_id, comment.id as comment_id, 301 | substring(story.title,1,20) as title,substring(comment.raw:text,1,20) as comment 302 | from story join comment on story.id=comment.root_id; 303 | ``` 304 | 305 | The docker compose file creates a dashboard for you that you can view in the [grafana dashboard locally](http://localhost:3000/dashboards). You can modify this and make your own. 306 | 307 | ![dashboard](https://github.com/bytewax/hacking-hacker-news/assets/6073079/e28f2ec1-d3cd-4f93-8237-340cb3638ea5) 308 | 309 | Have you made something interesting with this? We would love to see it! Share it on Reddit, Hacker News or your favorite social media platform and let us know :). 310 | 311 | If you think Bytewax and Proton are cool, show your support on GitHub with a star. 312 | 313 | ⭐ [Bytewax](https://github.com/bytewax/bytewax) 314 | 315 | ⭐ [Proton](https://github.com/timeplus-io/proton) 316 | 317 | 318 | -------------------------------------------------------------------------------- /compose.yaml: -------------------------------------------------------------------------------- 1 | services: 2 | 3 | proton: 4 | image: ghcr.io/timeplus-io/proton:latest 5 | pull_policy: always 6 | ports: 7 | - 8463:8463 8 | - 3218:3218 9 | volumes: 10 | - ./init_sql:/docker-entrypoint-initdb.d 11 | healthcheck: 12 | test: wget --no-verbose --tries=1 --spider http://localhost:8123/?query=show+create+comment || exit 1 13 | interval: 5s 14 | timeout: 10s 15 | retries: 3 16 | start_period: 10s 17 | 18 | hn_stream: 19 | build: 20 | context: . 21 | image: timeplus/hackernews_bytewax:latest 22 | environment: 23 | - PROTON_HOST=proton 24 | depends_on: 25 | proton: 26 | condition: service_healthy 27 | 28 | grafana: 29 | image: grafana/grafana:latest 30 | pull_policy: always 31 | ports: 32 | - 3000:3000 33 | environment: 34 | GF_AUTH_ANONYMOUS_ENABLED: 1 35 | GF_AUTH_ANONYMOUS_ORG_ROLE: Admin 36 | GF_INSTALL_PLUGINS: timeplus-proton-datasource 37 | GF_LOG_LEVEL: error 38 | volumes: 39 | - ./grafana_provisioning:/etc/grafana/provisioning 40 | depends_on: 41 | - hn_stream 42 | -------------------------------------------------------------------------------- /grafana_provisioning/dashboards/grafana_dashboard.json: -------------------------------------------------------------------------------- 1 | { 2 | "annotations": { 3 | "list": [ 4 | { 5 | "builtIn": 1, 6 | "datasource": { 7 | "type": "grafana", 8 | "uid": "-- Grafana --" 9 | }, 10 | "enable": true, 11 | "hide": true, 12 | "iconColor": "rgba(0, 211, 255, 1)", 13 | "name": "Annotations & Alerts", 14 | "type": "dashboard" 15 | } 16 | ] 17 | }, 18 | "editable": true, 19 | "fiscalYearStartMonth": 0, 20 | "graphTooltip": 0, 21 | "id": 1, 22 | "links": [], 23 | "liveNow": false, 24 | "panels": [ 25 | { 26 | "datasource": { 27 | "type": "timeplus-proton-datasource", 28 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 29 | }, 30 | "fieldConfig": { 31 | "defaults": { 32 | "color": { 33 | "mode": "thresholds" 34 | }, 35 | "mappings": [], 36 | "thresholds": { 37 | "mode": "absolute", 38 | "steps": [ 39 | { 40 | "color": "green", 41 | "value": null 42 | } 43 | ] 44 | } 45 | }, 46 | "overrides": [] 47 | }, 48 | "gridPos": { 49 | "h": 8, 50 | "w": 6, 51 | "x": 0, 52 | "y": 0 53 | }, 54 | "id": 4, 55 | "options": { 56 | "colorMode": "value", 57 | "graphMode": "area", 58 | "justifyMode": "auto", 59 | "orientation": "auto", 60 | "reduceOptions": { 61 | "calcs": [ 62 | "lastNotNull" 63 | ], 64 | "fields": "", 65 | "values": false 66 | }, 67 | "textMode": "auto", 68 | "wideLayout": true 69 | }, 70 | "pluginVersion": "10.2.3", 71 | "targets": [ 72 | { 73 | "addNow": false, 74 | "datasource": { 75 | "type": "timeplus-proton-datasource", 76 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 77 | }, 78 | "queryText": "select count() from story", 79 | "refId": "A" 80 | } 81 | ], 82 | "title": "# of Stories", 83 | "type": "stat" 84 | }, 85 | { 86 | "datasource": { 87 | "type": "timeplus-proton-datasource", 88 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 89 | }, 90 | "fieldConfig": { 91 | "defaults": { 92 | "color": { 93 | "mode": "thresholds" 94 | }, 95 | "mappings": [], 96 | "thresholds": { 97 | "mode": "absolute", 98 | "steps": [ 99 | { 100 | "color": "green", 101 | "value": null 102 | } 103 | ] 104 | } 105 | }, 106 | "overrides": [] 107 | }, 108 | "gridPos": { 109 | "h": 8, 110 | "w": 6, 111 | "x": 6, 112 | "y": 0 113 | }, 114 | "id": 5, 115 | "options": { 116 | "colorMode": "value", 117 | "graphMode": "area", 118 | "justifyMode": "auto", 119 | "orientation": "auto", 120 | "reduceOptions": { 121 | "calcs": [ 122 | "lastNotNull" 123 | ], 124 | "fields": "", 125 | "values": false 126 | }, 127 | "textMode": "auto", 128 | "wideLayout": true 129 | }, 130 | "pluginVersion": "10.2.3", 131 | "targets": [ 132 | { 133 | "addNow": false, 134 | "datasource": { 135 | "type": "timeplus-proton-datasource", 136 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 137 | }, 138 | "queryText": "select count() from comment", 139 | "refId": "A" 140 | } 141 | ], 142 | "title": "# of Comments", 143 | "type": "stat" 144 | }, 145 | { 146 | "datasource": { 147 | "type": "timeplus-proton-datasource", 148 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 149 | }, 150 | "fieldConfig": { 151 | "defaults": { 152 | "color": { 153 | "mode": "palette-classic" 154 | }, 155 | "custom": { 156 | "axisBorderShow": false, 157 | "axisCenteredZero": false, 158 | "axisColorMode": "text", 159 | "axisLabel": "", 160 | "axisPlacement": "auto", 161 | "fillOpacity": 80, 162 | "gradientMode": "none", 163 | "hideFrom": { 164 | "legend": false, 165 | "tooltip": false, 166 | "viz": false 167 | }, 168 | "lineWidth": 1, 169 | "scaleDistribution": { 170 | "type": "linear" 171 | }, 172 | "thresholdsStyle": { 173 | "mode": "off" 174 | } 175 | }, 176 | "mappings": [], 177 | "thresholds": { 178 | "mode": "absolute", 179 | "steps": [ 180 | { 181 | "color": "green", 182 | "value": null 183 | }, 184 | { 185 | "color": "red", 186 | "value": 80 187 | } 188 | ] 189 | } 190 | }, 191 | "overrides": [] 192 | }, 193 | "gridPos": { 194 | "h": 8, 195 | "w": 12, 196 | "x": 12, 197 | "y": 0 198 | }, 199 | "id": 6, 200 | "options": { 201 | "barRadius": 0, 202 | "barWidth": 0.97, 203 | "colorByField": "emit_version()", 204 | "fullHighlight": false, 205 | "groupWidth": 0.7, 206 | "legend": { 207 | "calcs": [], 208 | "displayMode": "list", 209 | "placement": "bottom", 210 | "showLegend": false 211 | }, 212 | "orientation": "auto", 213 | "showValue": "auto", 214 | "stacking": "none", 215 | "tooltip": { 216 | "mode": "single", 217 | "sort": "none" 218 | }, 219 | "xField": "by", 220 | "xTickLabelRotation": 0, 221 | "xTickLabelSpacing": 0 222 | }, 223 | "targets": [ 224 | { 225 | "addNow": false, 226 | "datasource": { 227 | "type": "timeplus-proton-datasource", 228 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 229 | }, 230 | "queryText": "select `by`,count() as comments,emit_version() from comment where `by`<>'' group by `by` order by comments desc limit 5 by emit_version()", 231 | "refId": "A" 232 | } 233 | ], 234 | "title": "Most active commenters", 235 | "transformations": [ 236 | { 237 | "id": "sortBy", 238 | "options": { 239 | "fields": {}, 240 | "sort": [ 241 | { 242 | "desc": true, 243 | "field": "emit_version()" 244 | } 245 | ] 246 | } 247 | }, 248 | { 249 | "id": "limit", 250 | "options": { 251 | "limitField": 5 252 | } 253 | } 254 | ], 255 | "type": "barchart" 256 | }, 257 | { 258 | "datasource": { 259 | "type": "timeplus-proton-datasource", 260 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 261 | }, 262 | "fieldConfig": { 263 | "defaults": { 264 | "color": { 265 | "mode": "thresholds" 266 | }, 267 | "custom": { 268 | "align": "auto", 269 | "cellOptions": { 270 | "type": "auto" 271 | }, 272 | "inspect": false 273 | }, 274 | "mappings": [], 275 | "thresholds": { 276 | "mode": "absolute", 277 | "steps": [ 278 | { 279 | "color": "green", 280 | "value": null 281 | }, 282 | { 283 | "color": "red", 284 | "value": 80 285 | } 286 | ] 287 | } 288 | }, 289 | "overrides": [ 290 | { 291 | "matcher": { 292 | "id": "byName", 293 | "options": "id" 294 | }, 295 | "properties": [ 296 | { 297 | "id": "custom.width", 298 | "value": 97 299 | } 300 | ] 301 | }, 302 | { 303 | "matcher": { 304 | "id": "byName", 305 | "options": "_tp_time" 306 | }, 307 | "properties": [ 308 | { 309 | "id": "custom.width", 310 | "value": 159 311 | } 312 | ] 313 | }, 314 | { 315 | "matcher": { 316 | "id": "byName", 317 | "options": "by" 318 | }, 319 | "properties": [ 320 | { 321 | "id": "custom.width", 322 | "value": 113 323 | } 324 | ] 325 | } 326 | ] 327 | }, 328 | "gridPos": { 329 | "h": 8, 330 | "w": 12, 331 | "x": 0, 332 | "y": 8 333 | }, 334 | "id": 1, 335 | "options": { 336 | "cellHeight": "sm", 337 | "footer": { 338 | "countRows": true, 339 | "fields": "", 340 | "reducer": [ 341 | "count" 342 | ], 343 | "show": true 344 | }, 345 | "showHeader": true, 346 | "sortBy": [] 347 | }, 348 | "pluginVersion": "10.2.3", 349 | "targets": [ 350 | { 351 | "addNow": false, 352 | "datasource": { 353 | "type": "timeplus-proton-datasource", 354 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 355 | }, 356 | "queryText": "select _tp_time,id,`by`,title from story", 357 | "refId": "A" 358 | } 359 | ], 360 | "title": "Stories", 361 | "transformations": [ 362 | { 363 | "id": "sortBy", 364 | "options": { 365 | "fields": {}, 366 | "sort": [ 367 | { 368 | "desc": true, 369 | "field": "_tp_time" 370 | } 371 | ] 372 | } 373 | } 374 | ], 375 | "type": "table" 376 | }, 377 | { 378 | "datasource": { 379 | "type": "timeplus-proton-datasource", 380 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 381 | }, 382 | "fieldConfig": { 383 | "defaults": { 384 | "color": { 385 | "mode": "thresholds" 386 | }, 387 | "custom": { 388 | "align": "auto", 389 | "cellOptions": { 390 | "type": "auto" 391 | }, 392 | "inspect": false 393 | }, 394 | "mappings": [], 395 | "thresholds": { 396 | "mode": "absolute", 397 | "steps": [ 398 | { 399 | "color": "green", 400 | "value": null 401 | }, 402 | { 403 | "color": "red", 404 | "value": 80 405 | } 406 | ] 407 | } 408 | }, 409 | "overrides": [ 410 | { 411 | "matcher": { 412 | "id": "byName", 413 | "options": "id" 414 | }, 415 | "properties": [ 416 | { 417 | "id": "custom.width", 418 | "value": 95 419 | } 420 | ] 421 | }, 422 | { 423 | "matcher": { 424 | "id": "byName", 425 | "options": "_tp_time" 426 | }, 427 | "properties": [ 428 | { 429 | "id": "custom.width", 430 | "value": 190 431 | } 432 | ] 433 | }, 434 | { 435 | "matcher": { 436 | "id": "byName", 437 | "options": "root_id" 438 | }, 439 | "properties": [ 440 | { 441 | "id": "custom.width", 442 | "value": 95 443 | } 444 | ] 445 | }, 446 | { 447 | "matcher": { 448 | "id": "byName", 449 | "options": "by" 450 | }, 451 | "properties": [ 452 | { 453 | "id": "custom.width", 454 | "value": 106 455 | } 456 | ] 457 | } 458 | ] 459 | }, 460 | "gridPos": { 461 | "h": 8, 462 | "w": 12, 463 | "x": 12, 464 | "y": 8 465 | }, 466 | "id": 2, 467 | "options": { 468 | "cellHeight": "sm", 469 | "footer": { 470 | "countRows": true, 471 | "fields": "", 472 | "reducer": [ 473 | "count" 474 | ], 475 | "show": true 476 | }, 477 | "showHeader": true, 478 | "sortBy": [] 479 | }, 480 | "pluginVersion": "10.2.3", 481 | "targets": [ 482 | { 483 | "addNow": false, 484 | "datasource": { 485 | "type": "timeplus-proton-datasource", 486 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 487 | }, 488 | "queryText": "select _tp_time,id,root_id,`by`,raw:text as text from comment", 489 | "refId": "A" 490 | } 491 | ], 492 | "title": "Comments", 493 | "transformations": [ 494 | { 495 | "id": "sortBy", 496 | "options": { 497 | "fields": {}, 498 | "sort": [ 499 | { 500 | "desc": true, 501 | "field": "_tp_time" 502 | } 503 | ] 504 | } 505 | } 506 | ], 507 | "type": "table" 508 | }, 509 | { 510 | "datasource": { 511 | "type": "timeplus-proton-datasource", 512 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 513 | }, 514 | "fieldConfig": { 515 | "defaults": { 516 | "color": { 517 | "mode": "thresholds" 518 | }, 519 | "custom": { 520 | "align": "auto", 521 | "cellOptions": { 522 | "type": "auto" 523 | }, 524 | "inspect": false 525 | }, 526 | "mappings": [], 527 | "thresholds": { 528 | "mode": "absolute", 529 | "steps": [ 530 | { 531 | "color": "green", 532 | "value": null 533 | }, 534 | { 535 | "color": "red", 536 | "value": 80 537 | } 538 | ] 539 | } 540 | }, 541 | "overrides": [ 542 | { 543 | "matcher": { 544 | "id": "byName", 545 | "options": "id" 546 | }, 547 | "properties": [ 548 | { 549 | "id": "custom.width", 550 | "value": 95 551 | } 552 | ] 553 | }, 554 | { 555 | "matcher": { 556 | "id": "byName", 557 | "options": "_tp_time" 558 | }, 559 | "properties": [ 560 | { 561 | "id": "custom.width", 562 | "value": 163 563 | } 564 | ] 565 | }, 566 | { 567 | "matcher": { 568 | "id": "byName", 569 | "options": "root_id" 570 | }, 571 | "properties": [ 572 | { 573 | "id": "custom.width", 574 | "value": 95 575 | } 576 | ] 577 | }, 578 | { 579 | "matcher": { 580 | "id": "byName", 581 | "options": "story_time" 582 | }, 583 | "properties": [ 584 | { 585 | "id": "custom.width", 586 | "value": 182 587 | } 588 | ] 589 | }, 590 | { 591 | "matcher": { 592 | "id": "byName", 593 | "options": "comment_time" 594 | }, 595 | "properties": [ 596 | { 597 | "id": "custom.width", 598 | "value": 189 599 | } 600 | ] 601 | }, 602 | { 603 | "matcher": { 604 | "id": "byName", 605 | "options": "story_id" 606 | }, 607 | "properties": [ 608 | { 609 | "id": "custom.width", 610 | "value": 127 611 | } 612 | ] 613 | }, 614 | { 615 | "matcher": { 616 | "id": "byName", 617 | "options": "comment_id" 618 | }, 619 | "properties": [ 620 | { 621 | "id": "custom.width", 622 | "value": 129 623 | } 624 | ] 625 | }, 626 | { 627 | "matcher": { 628 | "id": "byName", 629 | "options": "title" 630 | }, 631 | "properties": [ 632 | { 633 | "id": "custom.width", 634 | "value": 325 635 | } 636 | ] 637 | } 638 | ] 639 | }, 640 | "gridPos": { 641 | "h": 7, 642 | "w": 24, 643 | "x": 0, 644 | "y": 16 645 | }, 646 | "id": 3, 647 | "options": { 648 | "cellHeight": "sm", 649 | "footer": { 650 | "countRows": false, 651 | "fields": "", 652 | "reducer": [ 653 | "sum" 654 | ], 655 | "show": false 656 | }, 657 | "showHeader": true, 658 | "sortBy": [] 659 | }, 660 | "pluginVersion": "10.2.3", 661 | "targets": [ 662 | { 663 | "addNow": false, 664 | "datasource": { 665 | "type": "timeplus-proton-datasource", 666 | "uid": "c24e0faf-1490-4321-a373-7b2b07ca2e38" 667 | }, 668 | "queryText": "select \n story._tp_time as story_time,comment._tp_time as comment_time,\n story.id as story_id, comment.id as comment_id,\n story.title as title,comment.raw:text as comment\nfrom story join comment on story.id=comment.root_id", 669 | "refId": "A" 670 | } 671 | ], 672 | "title": "Stories&Comments", 673 | "type": "table" 674 | } 675 | ], 676 | "refresh": "", 677 | "schemaVersion": 39, 678 | "tags": [], 679 | "templating": { 680 | "list": [] 681 | }, 682 | "time": { 683 | "from": "now-6h", 684 | "to": "now" 685 | }, 686 | "timepicker": {}, 687 | "timezone": "", 688 | "title": "Hackernews Live Dashboard", 689 | "uid": "d299c092-7bda-4a37-ba1d-5f484fa4db72", 690 | "version": 5, 691 | "weekStart": "" 692 | } -------------------------------------------------------------------------------- /grafana_provisioning/dashboards/main.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: 1 2 | 3 | providers: 4 | # an unique provider name. Required 5 | - name: 'a unique provider name' 6 | # Org id. Default to 1 7 | orgId: 1 8 | # name of the dashboard folder. 9 | folder: '' 10 | # folder UID. will be automatically generated if not specified 11 | folderUid: '' 12 | # provider type. Default to 'file' 13 | type: file 14 | # disable dashboard deletion 15 | disableDeletion: false 16 | # how often Grafana will scan for changed dashboards 17 | updateIntervalSeconds: 10 18 | # allow updating provisioned dashboards from the UI 19 | allowUiUpdates: true 20 | options: 21 | # path to dashboard files on disk. Required when using the 'file' type 22 | path: /etc/grafana/provisioning/dashboards 23 | # use folder names from filesystem to create folders in Grafana 24 | foldersFromFilesStructure: true -------------------------------------------------------------------------------- /grafana_provisioning/datasources/automatic.yml: -------------------------------------------------------------------------------- 1 | apiVersion: 1 2 | 3 | datasources: 4 | - name: Proton 5 | type: timeplus-proton-datasource 6 | uid: c24e0faf-1490-4321-a373-7b2b07ca2e38 7 | typeName: Proton 8 | access: proxy 9 | url: 10 | password: 11 | user: 12 | database: 13 | isDefault: true 14 | basicAuth: false 15 | basicAuthUser: 16 | basicAuthPassword: 17 | withCredentials: 18 | jsonData: 19 | host: proton 20 | readOnly: false 21 | secureJsonData: 22 | tlsCACert: "" 23 | tlsClientCert: "" 24 | tlsClientKey: "" 25 | version: 1 26 | editable: true -------------------------------------------------------------------------------- /hackernews.py: -------------------------------------------------------------------------------- 1 | import logging 2 | from datetime import timedelta 3 | from typing import Optional, Tuple 4 | import os 5 | import json 6 | 7 | import requests 8 | from bytewax import operators as op 9 | from bytewax.dataflow import Dataflow 10 | from bytewax.inputs import SimplePollingSource 11 | 12 | from proton import ProtonSink 13 | 14 | logging.basicConfig(level=logging.INFO) 15 | logger = logging.getLogger(__name__) 16 | 17 | 18 | class HNSource(SimplePollingSource): 19 | def next_item(self): 20 | return ( 21 | "GLOBAL_ID", 22 | requests.get("https://hacker-news.firebaseio.com/v0/maxitem.json").json(), 23 | ) 24 | 25 | 26 | def get_id_stream(old_max_id, new_max_id) -> Tuple[str,list]: 27 | if old_max_id is None: 28 | # Get the last 150 items on the first run. 29 | old_max_id = new_max_id - 150 30 | return (new_max_id, range(old_max_id, new_max_id)) 31 | 32 | 33 | def download_metadata(hn_id) -> Optional[Tuple[str, dict]]: 34 | # Given an hacker news id returned from the api, fetch metadata 35 | # Try 3 times, waiting more and more, or give up 36 | data = requests.get( 37 | f"https://hacker-news.firebaseio.com/v0/item/{hn_id}.json" 38 | ).json() 39 | 40 | if data is None: 41 | logger.warning(f"Couldn't fetch item {hn_id}, skipping") 42 | return None 43 | return (str(hn_id), data) 44 | 45 | 46 | def recurse_tree(metadata, og_metadata=None) -> any: 47 | if not og_metadata: 48 | og_metadata = metadata 49 | try: 50 | parent_id = metadata["parent"] 51 | parent_metadata = download_metadata(parent_id) 52 | return recurse_tree(parent_metadata[1], og_metadata) 53 | except KeyError: 54 | return (metadata["id"], 55 | { 56 | **og_metadata, 57 | "root_id":metadata["id"] 58 | } 59 | ) 60 | 61 | 62 | def key_on_parent(key__metadata) -> tuple: 63 | key, metadata = recurse_tree(key__metadata[1]) 64 | return (str(key), metadata) 65 | 66 | 67 | def format(id__metadata): 68 | id, metadata = id__metadata 69 | return json.dumps(metadata) 70 | 71 | flow = Dataflow("hn_scraper") 72 | max_id = op.input("in", flow, HNSource(timedelta(seconds=15))) 73 | id_stream = op.stateful_map("range", max_id, lambda: None, get_id_stream).then( 74 | op.flat_map, "strip_key_flatten", lambda key_ids: key_ids[1]).then( 75 | op.redistribute, "redist") 76 | id_stream = op.filter_map("meta_download", id_stream, download_metadata) 77 | split_stream = op.branch("split_comments", id_stream, lambda item: item[1]["type"] == "story") 78 | story_stream = split_stream.trues 79 | story_stream = op.map("format_stories", story_stream, format) 80 | comment_stream = split_stream.falses 81 | comment_stream = op.map("key_on_parent", comment_stream, key_on_parent) 82 | comment_stream = op.map("format_comments", comment_stream, format) 83 | op.inspect("stories", story_stream) 84 | op.inspect("comments", comment_stream) 85 | op.output("stories-out", story_stream, ProtonSink("hn_stories_raw", os.environ.get("PROTON_HOST","127.0.0.1"))) 86 | op.output("comments-out", comment_stream, ProtonSink("hn_comments_raw", os.environ.get("PROTON_HOST","127.0.0.1"))) 87 | -------------------------------------------------------------------------------- /init_sql/init.sql: -------------------------------------------------------------------------------- 1 | CREATE STREAM IF NOT EXISTS hn_stories_raw(raw string); 2 | 3 | CREATE STREAM IF NOT EXISTS hn_comments_raw(raw string); 4 | 5 | CREATE MATERIALIZED VIEW IF NOT EXISTS hn_stories AS 6 | SELECT to_time(raw:time) AS _tp_time,raw:id::int AS id,raw:title AS title,raw:by AS by, raw FROM hn_stories_raw; 7 | 8 | CREATE MATERIALIZED VIEW IF NOT EXISTS hn_comments AS 9 | SELECT to_time(raw:time) AS _tp_time,raw:id::int AS id,raw:root_id::int AS root_id,raw:by AS by, raw FROM hn_comments_raw; 10 | 11 | CREATE VIEW IF NOT EXISTS story AS SELECT * FROM hn_stories WHERE _tp_time>earliest_ts(); 12 | 13 | CREATE VIEW IF NOT EXISTS comment AS SELECT * FROM hn_comments WHERE _tp_time>earliest_ts(); -------------------------------------------------------------------------------- /proton.py: -------------------------------------------------------------------------------- 1 | """Output to Timeplus Proton.""" 2 | from bytewax.outputs import DynamicSink, StatelessSinkPartition 3 | from proton_driver import client 4 | import logging 5 | 6 | __all__ = [ 7 | "ProtonSink", 8 | ] 9 | logger = logging.getLogger(__name__) 10 | logger.setLevel(logging.INFO) 11 | 12 | class _ProtonSinkPartition(StatelessSinkPartition): 13 | def __init__(self, stream: str, host: str): 14 | self.client=client.Client(host=host, port=8463) 15 | self.stream=stream 16 | sql=f"CREATE STREAM IF NOT EXISTS `{stream}` (raw string)" 17 | logger.debug(sql) 18 | self.client.execute(sql) 19 | 20 | def write_batch(self, items): 21 | logger.debug(f"inserting data {items}") 22 | rows=[] 23 | for item in items: 24 | rows.append([item]) # single column in each row 25 | sql = f"INSERT INTO `{self.stream}` (raw) VALUES" 26 | logger.debug(f"inserting data {sql}") 27 | self.client.execute(sql,rows) 28 | 29 | class ProtonSink(DynamicSink): 30 | def __init__(self, stream: str, host: str): 31 | self.stream = stream 32 | self.host = host if host is not None and host != "" else "127.0.0.1" 33 | 34 | """Write each output item to Proton on that worker. 35 | 36 | Items consumed from the dataflow must look like a string. Use a 37 | proceeding map step to do custom formatting. 38 | 39 | Workers are the unit of parallelism. 40 | 41 | Can support at-least-once processing. Messages from the resume 42 | epoch will be duplicated right after resume. 43 | 44 | """ 45 | 46 | def build(self, worker_index, worker_count): 47 | """See ABC docstring.""" 48 | return _ProtonSinkPartition(self.stream, self.host) -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | bytewax==0.18 2 | requests 3 | proton-driver --------------------------------------------------------------------------------