dev2
This commit is contained in:
parent
cae3ccb119
commit
fd0de2d1c0
|
|
@ -27,4 +27,7 @@ testing/udo_test.py
|
|||
|
||||
Simulation.md
|
||||
|
||||
monkeytype.sqlite3
|
||||
monkeytype.sqlite3
|
||||
|
||||
distributed_produce/bash/
|
||||
distributed_produce/notes.txt
|
||||
175
README.md
175
README.md
|
|
@ -6,168 +6,21 @@
|
|||
\___/\__,_/\__,_/\____/_/ |_/_____/
|
||||
by BlockScience
|
||||
```
|
||||
***cadCAD*** is a Python package that assists in the processes of designing, testing and validating complex systems
|
||||
through simulation, with support for Monte Carlo methods, A/B testing and parameter sweeping.
|
||||
|
||||
**Introduction:**
|
||||
|
||||
***cadCAD*** is a Python library that assists in the processes of designing, testing and validating complex systems through
|
||||
simulation. At its core, cadCAD is a differential games engine that supports parameter sweeping and Monte Carlo analyses
|
||||
and can be easily integrated with other scientific computing Python modules and data science workflows.
|
||||
|
||||
**Description:**
|
||||
|
||||
cadCAD (complex adaptive systems computer-aided design) is a python based, unified modeling framework for stochastic
|
||||
dynamical systems and differential games for research, validation, and Computer Aided Design of economic systems created
|
||||
by BlockScience. It is capable of modeling systems at all levels of abstraction from Agent Based Modeling (ABM) to
|
||||
System Dynamics (SD), and enabling smooth integration of computational social science simulations with empirical data
|
||||
science workflows.
|
||||
|
||||
|
||||
An economic system is treated as a state-based model and defined through a set of endogenous and exogenous state
|
||||
variables which are updated through mechanisms and environmental processes, respectively. Behavioral models, which may
|
||||
be deterministic or stochastic, provide the evolution of the system within the action space of the mechanisms.
|
||||
Mathematical formulations of these economic games treat agent utility as derived from the state rather than direct from
|
||||
an action, creating a rich, dynamic modeling framework. Simulations may be run with a range of initial conditions and
|
||||
parameters for states, behaviors, mechanisms, and environmental processes to understand and visualize network behavior
|
||||
under various conditions. Support for A/B testing policies, Monte Carlo analysis, and other common numerical methods is
|
||||
provided.
|
||||
|
||||
|
||||
For example, cadCAD tool allows us to represent a company’s or community’s current business model along with a desired
|
||||
future state and helps make informed, rigorously tested decisions on how to get from today’s stage to the future state.
|
||||
It allows us to use code to solidify our conceptualized ideas and see if the outcome meets our expectations. We can
|
||||
iteratively refine our work until we have constructed a model that closely reflects reality at the start of the model,
|
||||
and see how it evolves. We can then use these results to inform business decisions.
|
||||
|
||||
#### Documentation:
|
||||
* ##### [Tutorials](tutorials)
|
||||
* ##### [System Model Configuration](documentation/Simulation_Configuration.md)
|
||||
* ##### [System Simulation Execution](documentation/Simulation_Execution.md)
|
||||
* ##### [Policy Aggregation](documentation/Policy_Aggregation.md)
|
||||
* ##### [System Model Parameter Sweep](documentation/System_Model_Parameter_Sweep.md)
|
||||
|
||||
#### 0. Installation:
|
||||
|
||||
**Python 3.6.5** :: Anaconda, Inc.
|
||||
|
||||
**Option A:** Build From Source
|
||||
# Getting Started
|
||||
## 1. Install cadCAD
|
||||
cadCAD requires [Python 3](https://www.python.org/downloads/)
|
||||
cadCAD can be installed using Python’s package manager, [pip](https://pypi.org/project/cadCAD/)
|
||||
```bash
|
||||
pip3 install -r requirements.txt
|
||||
python3 setup.py sdist bdist_wheel
|
||||
pip3 install dist/*.whl
|
||||
pip install cadCAD
|
||||
```
|
||||
## 2. Learn the basics
|
||||
Check out our tutorials (available both as [Jupyter Notebooks](tutorials) and
|
||||
[videos](https://www.youtube.com/watch?v=uJEiYHRWA9g&list=PLmWm8ksQq4YKtdRV-SoinhV6LbQMgX1we)) to familiarize yourself
|
||||
with some system modelling concepts and cadCAD terminology. Alternatively, go straight to the
|
||||
[documentation](documentation).
|
||||
|
||||
**Option B:** Proprietary Build Access
|
||||
|
||||
***IMPORTANT NOTE:*** Tokens are issued to those with access to proprietary builds of cadCAD and BlockScience employees **ONLY**.
|
||||
Replace \<TOKEN\> with an issued token in the script below.
|
||||
```bash
|
||||
pip3 install pandas pathos fn funcy tabulate
|
||||
pip3 install cadCAD --extra-index-url https://<TOKEN>@repo.fury.io/blockscience/
|
||||
```
|
||||
|
||||
|
||||
#### 1. [Configure System Model](documentation/Simulation_Configuration.md)
|
||||
|
||||
#### 2. [Execute Simulations:](documentation/Simulation_Execution.md)
|
||||
|
||||
##### Single Process Execution:
|
||||
Example System Model Configurations:
|
||||
* [System Model A](documentation/examples/sys_model_A.py):
|
||||
`/documentation/examples/sys_model_A.py`
|
||||
* [System Model B](documentation/examples/sys_model_B.py):
|
||||
`/documentation/examples/sys_model_B.py`
|
||||
|
||||
Example Simulation Executions:
|
||||
* [System Model A](documentation/examples/sys_model_A_exec.py):
|
||||
`/documentation/examples/sys_model_A_exec.py`
|
||||
* [System Model B](documentation/examples/sys_model_B_exec.py):
|
||||
`/documentation/examples/sys_model_B_exec.py`
|
||||
|
||||
```python
|
||||
import pandas as pd
|
||||
from tabulate import tabulate
|
||||
from cadCAD.engine import ExecutionMode, ExecutionContext, Executor
|
||||
from documentation.examples import sys_model_A
|
||||
from cadCAD import configs
|
||||
|
||||
exec_mode = ExecutionMode()
|
||||
|
||||
# Single Process Execution using a Single System Model Configuration:
|
||||
# sys_model_A
|
||||
sys_model_A = [configs[0]] # sys_model_A
|
||||
single_proc_ctx = ExecutionContext(context=exec_mode.single_proc)
|
||||
sys_model_A_simulation = Executor(exec_context=single_proc_ctx, configs=sys_model_A)
|
||||
|
||||
sys_model_A_raw_result, sys_model_A_tensor_field = sys_model_A_simulation.execute()
|
||||
sys_model_A_result = pd.DataFrame(sys_model_A_raw_result)
|
||||
print()
|
||||
print("Tensor Field: sys_model_A")
|
||||
print(tabulate(sys_model_A_tensor_field, headers='keys', tablefmt='psql'))
|
||||
print("Result: System Events DataFrame")
|
||||
print(tabulate(sys_model_A_result, headers='keys', tablefmt='psql'))
|
||||
print()
|
||||
```
|
||||
|
||||
##### Multiple Simulations (Concurrent):
|
||||
###### Multiple Simulation Execution (Multi Process Execution)
|
||||
System Model Configurations:
|
||||
* [System Model A](documentation/examples/sys_model_A.py):
|
||||
`/documentation/examples/sys_model_A.py`
|
||||
* [System Model B](documentation/examples/sys_model_B.py):
|
||||
`/documentation/examples/sys_model_B.py`
|
||||
|
||||
[Example Simulation Executions:](documentation/examples/sys_model_AB_exec.py)
|
||||
`/documentation/examples/sys_model_AB_exec.py`
|
||||
|
||||
```python
|
||||
import pandas as pd
|
||||
from tabulate import tabulate
|
||||
from cadCAD.engine import ExecutionMode, ExecutionContext, Executor
|
||||
from documentation.examples import sys_model_A, sys_model_B
|
||||
from cadCAD import configs
|
||||
|
||||
exec_mode = ExecutionMode()
|
||||
|
||||
# # Multiple Processes Execution using Multiple System Model Configurations:
|
||||
# # sys_model_A & sys_model_B
|
||||
multi_proc_ctx = ExecutionContext(context=exec_mode.multi_proc)
|
||||
sys_model_AB_simulation = Executor(exec_context=multi_proc_ctx, configs=configs)
|
||||
|
||||
i = 0
|
||||
config_names = ['sys_model_A', 'sys_model_B']
|
||||
for sys_model_AB_raw_result, sys_model_AB_tensor_field in sys_model_AB_simulation.execute():
|
||||
sys_model_AB_result = pd.DataFrame(sys_model_AB_raw_result)
|
||||
print()
|
||||
print(f"Tensor Field: {config_names[i]}")
|
||||
print(tabulate(sys_model_AB_tensor_field, headers='keys', tablefmt='psql'))
|
||||
print("Result: System Events DataFrame:")
|
||||
print(tabulate(sys_model_AB_result, headers='keys', tablefmt='psql'))
|
||||
print()
|
||||
i += 1
|
||||
```
|
||||
|
||||
##### Parameter Sweep Simulation (Concurrent):
|
||||
[Example:](documentation/examples/param_sweep.py)
|
||||
`/documentation/examples/param_sweep.py`
|
||||
|
||||
```python
|
||||
import pandas as pd
|
||||
from tabulate import tabulate
|
||||
# The following imports NEED to be in the exact order
|
||||
from cadCAD.engine import ExecutionMode, ExecutionContext, Executor
|
||||
from documentation.examples import param_sweep
|
||||
from cadCAD import configs
|
||||
|
||||
exec_mode = ExecutionMode()
|
||||
multi_proc_ctx = ExecutionContext(context=exec_mode.multi_proc)
|
||||
run = Executor(exec_context=multi_proc_ctx, configs=configs)
|
||||
|
||||
for raw_result, tensor_field in run.execute():
|
||||
result = pd.DataFrame(raw_result)
|
||||
print()
|
||||
print("Tensor Field:")
|
||||
print(tabulate(tensor_field, headers='keys', tablefmt='psql'))
|
||||
print("Output:")
|
||||
print(tabulate(result, headers='keys', tablefmt='psql'))
|
||||
print()
|
||||
```
|
||||
## 3. Connect
|
||||
Find other cadCAD users at our [Discourse](https://community.cadcad.org/). We are a small but rapidly growing community.
|
||||
|
|
|
|||
|
|
@ -0,0 +1,30 @@
|
|||
from datetime import datetime
|
||||
|
||||
# ToDo: Model Async communication here
|
||||
# ToDo: Configuration of Input
|
||||
def configure_producer(msg, config):
|
||||
from kafka import KafkaProducer
|
||||
def send_messages(events):
|
||||
producer = KafkaProducer(**config)
|
||||
|
||||
start_timestamp = datetime.now()
|
||||
for event in range(events):
|
||||
producer.send('test', msg(event)).get()
|
||||
delta = datetime.now() - start_timestamp
|
||||
|
||||
return start_timestamp, delta.total_seconds()
|
||||
|
||||
return send_messages
|
||||
|
||||
# def action(step):
|
||||
# step += 1
|
||||
# percent = step / 10000
|
||||
# print(str(datetime.now()) + " - " + str(percent) + ": " + str(step))
|
||||
|
||||
# def configure_consumer(config, elemental_action: function):
|
||||
# from kafka import KafkaConsumer
|
||||
# def receive_messages():
|
||||
# consumer = KafkaConsumer(**config)
|
||||
# return [elemental_action(i) for i, message in enumerate(consumer)]
|
||||
#
|
||||
# return receive_messages
|
||||
|
|
@ -0,0 +1 @@
|
|||
from cadCAD.distroduce.executor.spark.jobs import distributed_produce
|
||||
|
|
@ -0,0 +1,34 @@
|
|||
from cadCAD.distroduce.configuration.kakfa import configure_producer
|
||||
from pyspark.context import SparkContext
|
||||
|
||||
ascii_art = r'''
|
||||
___ _ __ _ __ __ __
|
||||
/ _ \ (_)___ / /_ ____ (_)/ / __ __ / /_ ___ ___/ /
|
||||
/ // // /(_-</ __// __// // _ \/ // // __// -_)/ _ /
|
||||
/____//_//___/\__//_/ _/_//_.__/\_,_/ \__/ \__/ \_,_/
|
||||
/ _ \ ____ ___ ___/ /__ __ ____ ___
|
||||
/ ___// __// _ \/ _ // // // __// -_)
|
||||
/_/ /_/ \___/\_,_/ \_,_/ \__/ \__/
|
||||
by Joshua E. Jodesty
|
||||
'''
|
||||
|
||||
|
||||
def distributed_produce(
|
||||
sc: SparkContext,
|
||||
spark_run,
|
||||
sim_time,
|
||||
sim_runs,
|
||||
rdd_parts,
|
||||
parameterized_message,
|
||||
prod_config
|
||||
):
|
||||
print(ascii_art)
|
||||
message_counts = [sim_time] * sim_runs
|
||||
msg_rdd = sc.parallelize(message_counts).repartition(rdd_parts)
|
||||
parts = msg_rdd.getNumPartitions()
|
||||
print()
|
||||
print(f"RDD_{spark_run} - Partitions: {parts}")
|
||||
print()
|
||||
|
||||
produce = configure_producer(parameterized_message, prod_config)
|
||||
return msg_rdd.map(produce).collect()
|
||||
|
|
@ -0,0 +1,2 @@
|
|||
def flatten(l):
|
||||
return [item for sublist in l for item in sublist]
|
||||
|
|
@ -0,0 +1,32 @@
|
|||
```
|
||||
___ _ __ _ __ __ __
|
||||
/ _ \ (_)___ / /_ ____ (_)/ / __ __ / /_ ___ ___/ /
|
||||
/ // // /(_-</ __// __// // _ \/ // // __// -_)/ _ /
|
||||
/____//_//___/\__//_/ _/_//_.__/\_,_/ \__/ \__/ \_,_/
|
||||
/ _ \ ____ ___ ___/ /__ __ ____ ___
|
||||
/ ___// __// _ \/ _ // // // __// -_)
|
||||
/_/ /_/ \___/\_,_/ \_,_/ \__/ \__/
|
||||
by Joshua E. Jodesty
|
||||
|
||||
```
|
||||
|
||||
**Description:**
|
||||
Distributed Produce (**distroduce**) is a cadCAD feature leveraging Apache Spark and Apache Kafka to enable in-stream
|
||||
data processing application development and throughput benchmarking for Kafka clusters.
|
||||
|
||||
**Properties:**
|
||||
* enables cadCAD's user-defined simulation framework to publish events/messages to Kafka Clusters
|
||||
* enables scalable message publishing Kafka clusters by distributing simulated event/message creation and publishing on
|
||||
an EMR cluster using Spark and Kafka Producer
|
||||
|
||||
|
||||
#### Installation / Build From Source:
|
||||
```
|
||||
pip3 install -r requirements.txt
|
||||
zip -rq distributed_produce/dist/distroduce.zip cadCAD/distroduce/
|
||||
```
|
||||
|
||||
#### Usage:
|
||||
```
|
||||
spark-submit --py-files distributed_produce/dist/distroduce.zip distributed_produce/examples/event_bench/main.py
|
||||
```
|
||||
|
|
@ -0,0 +1,10 @@
|
|||
___ _ __ _ __ __ __
|
||||
/ _ \ (_)___ / /_ ____ (_)/ / __ __ / /_ ___ ___/ /
|
||||
/ // // /(_-</ __// __// // _ \/ // // __// -_)/ _ /
|
||||
/____//_//___/\__//_/ _/_//_.__/\_,_/ \__/ \__/ \_,_/
|
||||
/ _ \ ____ ___ ___/ /__ __ ____ ___
|
||||
/ ___// __// _ \/ _ // // // __// -_)
|
||||
/_/ /_/ \___/\_,_/ \_,_/ \__/ \__/
|
||||
by Joshua E. Jodesty
|
||||
|
||||
promulgate
|
||||
Binary file not shown.
|
|
@ -0,0 +1,11 @@
|
|||
from kafka import KafkaConsumer
|
||||
from datetime import datetime
|
||||
|
||||
|
||||
consumer = KafkaConsumer('test', bootstrap_servers=['localhost:9092'])
|
||||
|
||||
i = 1
|
||||
for message in consumer:
|
||||
percent = i / 10000
|
||||
print(str(datetime.now()) + " - " + str(percent) + ": " + str(i))
|
||||
i += 1
|
||||
|
|
@ -0,0 +1,58 @@
|
|||
from functools import reduce
|
||||
from datetime import timedelta
|
||||
from pprint import pprint
|
||||
import time
|
||||
from pathos.multiprocessing import ThreadPool as TPool
|
||||
|
||||
from cadCAD.utils import flatten
|
||||
from cadCAD.distroduce.executor import distributed_produce
|
||||
from distributed_produce.examples.event_bench.spark.session import spark_context as sc
|
||||
|
||||
|
||||
def main(sc, spark_runs, rdd_parts, sim_time, sim_runs, parameterized_message):
|
||||
publish_times, spark_job_times = [], []
|
||||
prod_config = {'bootstrap_servers': 'localhost:9092', 'acks': 0}
|
||||
exec_spark_job = lambda run: distributed_produce(
|
||||
sc, run, sim_time, sim_runs, rdd_parts, parameterized_message, prod_config
|
||||
)
|
||||
|
||||
job = 0
|
||||
with TPool(spark_runs) as t:
|
||||
start_time = time.time()
|
||||
publish_times.append(t.map(exec_spark_job, range(spark_runs)))
|
||||
spark_job_times.append({'job_num': job, 'job_time': time.time() - start_time})
|
||||
job += 1
|
||||
|
||||
publish_times = sorted(flatten(list(reduce(lambda a, b: a + b, publish_times))), key=lambda x: x[0])
|
||||
# publish_start_sec = list(set([time[0].second for time in publish_times]))
|
||||
publish_send_secs = [time[1] for time in publish_times]
|
||||
publish_send_dts = [time[0] for time in publish_times]
|
||||
|
||||
send_time = publish_send_dts[-1] - publish_send_dts[0] + timedelta(microseconds=1000000 * publish_send_secs[-1])
|
||||
sent_messages = sim_time * sim_runs
|
||||
print(f"Spark Job Times: ")
|
||||
pprint(spark_job_times)
|
||||
print()
|
||||
print(f"Messages per Second: {float(sent_messages) / (float(send_time.seconds) + float(send_time.microseconds/1000000))}")
|
||||
print(f"Sent Messages: {sent_messages}")
|
||||
print(f"Send Time (Seconds): {send_time}")
|
||||
print(f"Avg. Send Loop Duration: {sum(publish_send_secs)/(spark_runs*25)}")
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sc.setLogLevel("ERROR")
|
||||
spark_runs = 1
|
||||
sim_time = 400
|
||||
sim_runs = 25
|
||||
rdd_parts = 8
|
||||
|
||||
def msg(t):
|
||||
return str(f"*****************message_{t}").encode('utf-8')
|
||||
|
||||
main(sc, spark_runs, rdd_parts, sim_time, sim_runs, msg)
|
||||
|
||||
# prod_config = {'bootstrap_servers': 'localhost:9092', 'acks': 0, 'max_in_flight_requests_per_connection': 1, 'batch_size': 0, 'retries': 0}
|
||||
# print(f"Second Starts: {len(publish_start_sec)}")
|
||||
# print(f"Start Seconds: {publish_start_sec}")
|
||||
# pprint(publish_send_secs)
|
||||
# pprint(publish_times)
|
||||
|
|
@ -0,0 +1,15 @@
|
|||
from pyspark.sql import SparkSession
|
||||
from pyspark.context import SparkContext
|
||||
import os
|
||||
|
||||
os.environ['PYSPARK_PYTHON'] = '/usr/local/bin/python3'
|
||||
os.environ['PYSPARK_DRIVER_PYTHON'] = '/usr/local/bin/python3'
|
||||
|
||||
spark = SparkSession\
|
||||
.builder\
|
||||
.appName("DistributedProduce")\
|
||||
.getOrCreate()
|
||||
|
||||
spark_context: SparkContext = spark.sparkContext
|
||||
print(f"Spark UI: {spark_context.uiWebUrl}")
|
||||
print()
|
||||
|
|
@ -3,4 +3,6 @@ wheel
|
|||
pathos
|
||||
fn
|
||||
tabulate
|
||||
funcy
|
||||
funcy
|
||||
pyspark
|
||||
kafka-python
|
||||
Loading…
Reference in New Issue