Skip to content

Commit 2cb28c4

Browse files
committed
js2py to mini-racer
1 parent f300e59 commit 2cb28c4

File tree

3 files changed

+108
-87
lines changed

3 files changed

+108
-87
lines changed

sdks/python/apache_beam/yaml/yaml_mapping.py

Lines changed: 73 additions & 78 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,10 @@
2929
from typing import TypeVar
3030
from typing import Union
3131

32+
import json
33+
import threading
34+
import uuid
35+
3236
import apache_beam as beam
3337
from apache_beam.io.filesystems import FileSystems
3438
from apache_beam.portability.api import schema_pb2
@@ -53,13 +57,12 @@
5357
from apache_beam.yaml.yaml_errors import maybe_with_exception_handling_transform_fn
5458
from apache_beam.yaml.yaml_provider import dicts_to_rows
5559

56-
# Import js2py package if it exists
5760
try:
58-
import js2py
59-
from js2py.base import JsObjectWrapper
61+
from py_mini_racer import MiniRacer
6062
except ImportError:
61-
js2py = None
62-
JsObjectWrapper = object
63+
MiniRacer = None
64+
65+
_js_thread_funcs = {}
6366

6467
_str_expression_fields = {
6568
'AssignTimestamps': 'timestamp',
@@ -178,18 +181,7 @@ def _check_mapping_arguments(
178181
raise ValueError(f'{transform_name} cannot specify "name" without "path"')
179182

180183

181-
# js2py's JsObjectWrapper object has a self-referencing __dict__ property
182-
# that cannot be pickled without implementing the __getstate__ and
183-
# __setstate__ methods.
184-
class _CustomJsObjectWrapper(JsObjectWrapper):
185-
def __init__(self, js_obj):
186-
super().__init__(js_obj.__dict__['_obj'])
187-
188-
def __getstate__(self):
189-
return self.__dict__.copy()
190184

191-
def __setstate__(self, state):
192-
self.__dict__.update(state)
193185

194186

195187
# TODO(yaml) Improve type inferencing for JS UDF's
@@ -205,83 +197,86 @@ def py_value_to_js_dict(py_value):
205197
return py_value
206198

207199

200+
def js_to_py(obj):
201+
"""Converts mini-racer mapped objects to standard Python types.
202+
203+
This is needed because ctx.eval returns JSMappedObjectImpl and JSArrayImpl
204+
for JS objects and arrays, which are not picklable and would fail when Beam
205+
tries to serialize rows containing them. We also preserve datetime objects
206+
which are correctly produced by ctx.eval for JS Date objects.
207+
"""
208+
import datetime
209+
from collections import abc
210+
211+
type_name = type(obj).__name__
212+
if type_name == 'JSMappedObjectImpl':
213+
return {k: js_to_py(v) for k, v in dict(obj).items()}
214+
elif type_name == 'JSArrayImpl':
215+
return [js_to_py(v) for v in list(obj)]
216+
elif isinstance(obj, datetime.datetime):
217+
return obj
218+
elif isinstance(obj, dict):
219+
return {k: js_to_py(v) for k, v in obj.items()}
220+
elif not isinstance(obj, str) and isinstance(obj, abc.Iterable):
221+
return [js_to_py(v) for v in list(obj)]
222+
else:
223+
return obj
224+
225+
208226
# TODO(yaml) Consider adding optional language version parameter to support
209227
# ECMAScript 5 and 6
210228
def _expand_javascript_mapping_func(
211229
original_fields, expression=None, callable=None, path=None, name=None):
212230

213-
# Check for installed js2py package
214-
if js2py is None:
231+
if MiniRacer is None:
215232
raise ValueError(
216-
"Javascript mapping functions are not supported on"
217-
" Python 3.12 or later.")
218-
219-
# import remaining js2py objects
220-
from js2py import base
221-
from js2py.constructors import jsdate
222-
from js2py.internals import simplex
223-
224-
js_array_type = (
225-
base.PyJsArray,
226-
base.PyJsArrayBuffer,
227-
base.PyJsInt8Array,
228-
base.PyJsUint8Array,
229-
base.PyJsUint8ClampedArray,
230-
base.PyJsInt16Array,
231-
base.PyJsUint16Array,
232-
base.PyJsInt32Array,
233-
base.PyJsUint32Array,
234-
base.PyJsFloat32Array,
235-
base.PyJsFloat64Array)
236-
237-
def _js_object_to_py_object(obj):
238-
if isinstance(obj, (base.PyJsNumber, base.PyJsString, base.PyJsBoolean)):
239-
return base.to_python(obj)
240-
elif isinstance(obj, js_array_type):
241-
return [_js_object_to_py_object(value) for value in obj.to_list()]
242-
elif isinstance(obj, jsdate.PyJsDate):
243-
return obj.to_utc_dt()
244-
elif isinstance(obj, (base.PyJsNull, base.PyJsUndefined)):
245-
return None
246-
elif isinstance(obj, base.PyJsError):
247-
raise RuntimeError(obj['message'])
248-
elif isinstance(obj, base.PyJsObject):
249-
return {
250-
key: _js_object_to_py_object(value['value'])
251-
for (key, value) in obj.own.items()
252-
}
253-
elif isinstance(obj, base.JsObjectWrapper):
254-
return _js_object_to_py_object(obj._obj)
255-
256-
return obj
257-
258-
if expression:
259-
source = '\n'.join(['function(__row__) {'] + [
260-
f' {name} = __row__.{name}'
261-
for name in original_fields if name in expression
262-
] + [' return (' + expression + ')'] + ['}'])
263-
js_func = _CustomJsObjectWrapper(js2py.eval_js(source))
264-
265-
elif callable:
266-
js_func = _CustomJsObjectWrapper(js2py.eval_js(callable))
233+
"JavaScript mapping functions require the 'mini-racer' package to be installed.")
267234

268-
else:
235+
udf_code = None
236+
if path:
269237
if not path.endswith('.js'):
270238
raise ValueError(f'File "{path}" is not a valid .js file.')
271239
udf_code = FileSystems.open(path).read().decode()
272-
js = js2py.EvalJs()
273-
js.eval(udf_code)
274-
js_func = _CustomJsObjectWrapper(getattr(js, name))
240+
elif expression:
241+
udf_code = f"var func = (__row__) => {{ " + " ".join([
242+
f"const {n} = __row__.{n};"
243+
for n in original_fields if n in expression
244+
]) + f" return ({expression}); }}"
245+
elif callable:
246+
udf_code = f"var func = {callable}"
247+
248+
udf_key = str(uuid.uuid4())
275249

276250
def js_wrapper(row):
251+
tid = threading.get_ident()
252+
253+
global _js_thread_funcs
254+
# MiniRacer contexts are not picklable and cannot be shared across threads.
255+
# We use a global dict keyed by thread ID to lazily create and cache a
256+
# context per thread.
257+
if tid not in _js_thread_funcs:
258+
_js_thread_funcs[tid] = {}
259+
260+
if udf_key not in _js_thread_funcs[tid]:
261+
ctx = MiniRacer()
262+
ctx.eval(udf_code)
263+
# We use ctx.eval instead of ctx.call to ensure that JavaScript Date
264+
# objects are correctly returned as Python datetime objects.
265+
# We JSON-serialize the arguments to pass them safely to eval.
266+
if expression or callable:
267+
_js_thread_funcs[tid][udf_key] = lambda x: ctx.eval(f"func({json.dumps(x)})")
268+
else:
269+
_js_thread_funcs[tid][udf_key] = lambda x: ctx.eval(f"{name}({json.dumps(x)})")
270+
271+
func = _js_thread_funcs[tid][udf_key]
277272
row_as_dict = py_value_to_js_dict(row)
278273
try:
279-
js_result = js_func(row_as_dict)
280-
except simplex.JsException as exn:
274+
result = func(row_as_dict)
275+
except Exception as exn:
281276
raise RuntimeError(
282-
f"Error evaluating javascript expression: "
283-
f"{exn.mes['message']}") from exn
284-
return dicts_to_rows(_js_object_to_py_object(js_result))
277+
f"Error evaluating JavaScript expression: {exn}") from exn
278+
result = js_to_py(result)
279+
return dicts_to_rows(result)
285280

286281
return js_wrapper
287282

sdks/python/apache_beam/yaml/yaml_udf_test.py

Lines changed: 34 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -32,10 +32,10 @@
3232
from apache_beam.yaml.yaml_transform import YamlTransform
3333

3434
try:
35-
import js2py
35+
from py_mini_racer import MiniRacer
3636
except ImportError:
37-
js2py = None
38-
logging.warning('js2py is not installed; some tests will be skipped.')
37+
MiniRacer = None
38+
logging.warning('py_mini_racer is not installed; some tests will be skipped.')
3939

4040

4141
def as_rows():
@@ -63,7 +63,7 @@ def setUp(self):
6363
def tearDown(self):
6464
shutil.rmtree(self.tmpdir)
6565

66-
@unittest.skipIf(js2py is None, 'js2py not installed.')
66+
@unittest.skipIf(MiniRacer is None, 'py_mini_racer not installed.')
6767
def test_map_to_fields_filter_inline_js(self):
6868
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
6969
pickle_library='cloudpickle', yaml_experimental_features=['javascript'
@@ -197,7 +197,7 @@ def test_map_to_fields_sql_reserved_keyword_append():
197197
beam.Row(label='389a', timestamp=2, label_copy="389a"),
198198
]))
199199

200-
@unittest.skipIf(js2py is None, 'js2py not installed.')
200+
@unittest.skipIf(MiniRacer is None, 'py_mini_racer not installed.')
201201
def test_filter_inline_js(self):
202202
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
203203
pickle_library='cloudpickle', yaml_experimental_features=['javascript'
@@ -252,7 +252,7 @@ def test_filter_inline_py(self):
252252
row=beam.Row(rank=2, values=[7, 8, 9])),
253253
]))
254254

255-
@unittest.skipIf(js2py is None, 'js2py not installed.')
255+
@unittest.skipIf(MiniRacer is None, 'py_mini_racer not installed.')
256256
def test_filter_expression_js(self):
257257
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
258258
pickle_library='cloudpickle', yaml_experimental_features=['javascript'
@@ -296,7 +296,7 @@ def test_filter_expression_py(self):
296296
row=beam.Row(rank=0, values=[1, 2, 3])),
297297
]))
298298

299-
@unittest.skipIf(js2py is None, 'js2py not installed.')
299+
@unittest.skipIf(MiniRacer is None, 'py_mini_racer not installed.')
300300
def test_filter_inline_js_file(self):
301301
data = '''
302302
function f(x) {
@@ -373,6 +373,33 @@ def g(x):
373373
conductor=389,
374374
row=beam.Row(rank=2, values=[7, 8, 9])),
375375
]))
376+
@unittest.skipIf(MiniRacer is None, 'py_mini_racer not installed.')
377+
def test_map_to_fields_js_date(self):
378+
import datetime
379+
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
380+
pickle_library='cloudpickle',
381+
yaml_experimental_features=['javascript'])) as p:
382+
elements = p | beam.Create([beam.Row(label='11a')])
383+
result = elements | YamlTransform(
384+
'''
385+
type: MapToFields
386+
config:
387+
language: javascript
388+
fields:
389+
date:
390+
callable: |
391+
function get_date(x) {
392+
return new Date('2026-04-17T18:00:00Z')
393+
}
394+
''')
395+
396+
expected_date = datetime.datetime(2026, 4, 17, 18, 0, 0, tzinfo=datetime.timezone.utc)
397+
398+
assert_that(
399+
result | as_rows(),
400+
equal_to([
401+
beam.Row(date=expected_date),
402+
]))
376403

377404

378405
if __name__ == '__main__':

sdks/python/setup.py

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -621,8 +621,7 @@ def get_portability_package_data():
621621
'docstring-parser>=0.15,<1.0',
622622
'jinja2>=3.0,<3.2',
623623
'virtualenv-clone>=0.5,<1.0',
624-
# https://github.com/PiotrDabkowski/Js2Py/issues/317
625-
'js2py>=0.74,<1; python_version<"3.12"',
624+
'mini-racer',
626625
'jsonschema>=4.0.0,<5.0.0',
627626
] + dataframe_dependency,
628627
# Keep the following dependencies in line with what we test against

0 commit comments

Comments
 (0)