diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index d6a418f3300..3c2c6428aed 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -50,3 +50,5 @@ tenacity==8.5.0 SQLAlchemy==2.0.37 pg8000==1.31.2 pympler==1.1 +tensorflow==2.19.0 +dataprofiler==0.13.3 \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto index 7e9c65b3a4a..761c1cf4d43 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto @@ -57,6 +57,7 @@ message ControlRequest { EmptyRequest emptyRequest = 56; PrepareCheckpointRequest prepareCheckpointRequest = 57; QueryStatisticsRequest queryStatisticsRequest = 58; + QueryTableProfileRequest queryTableProfileRequest = 59; // request for testing Ping ping = 100; @@ -271,4 +272,8 @@ message PrepareCheckpointRequest{ message QueryStatisticsRequest{ repeated core.ActorVirtualIdentity filterByWorkers = 1; +} + +message QueryTableProfileRequest{ + } \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto index 0ebb5bbf06b..0096495de65 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto @@ -19,6 +19,7 @@ syntax = "proto3"; package edu.uci.ics.amber.engine.architecture.rpc; import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; +import "edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto"; import "scalapb/scalapb.proto"; option (scalapb.options) = { @@ -43,6 +44,7 @@ message ControlReturn { WorkerStateResponse workerStateResponse = 50; WorkerMetricsResponse workerMetricsResponse = 51; FinalizeCheckpointResponse finalizeCheckpointResponse = 52; + TableProfileResponse tableProfileResponse = 53; // common responses ControlError controlError = 101; @@ -137,4 +139,8 @@ message WorkerStateResponse { message WorkerMetricsResponse { worker.WorkerMetrics metrics = 1 [(scalapb.field).no_box = true]; +} + +message TableProfileResponse { + worker.TableProfile table_profiles = 1 [(scalapb.field).no_box = true]; } \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto index cb99bf234be..e840fb9bfa9 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/rpc/workerservice.proto @@ -40,6 +40,7 @@ service WorkerService { rpc PauseWorker(EmptyRequest) returns (WorkerStateResponse); rpc PrepareCheckpoint(PrepareCheckpointRequest) returns (EmptyReturn); rpc QueryStatistics(EmptyRequest) returns (WorkerMetricsResponse); + rpc QueryTableProfile(EmptyRequest) returns (TableProfileResponse); rpc ResumeWorker(EmptyRequest) returns (WorkerStateResponse); rpc RetrieveState(EmptyRequest) returns (EmptyReturn); rpc RetryCurrentTuple(EmptyRequest) returns (EmptyReturn); diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto new file mode 100644 index 00000000000..e308d2ab0f7 --- /dev/null +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto @@ -0,0 +1,142 @@ +// 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. + +syntax = "proto3"; + +package edu.uci.ics.amber.engine.architecture.worker; + +import "scalapb/scalapb.proto"; + +option (scalapb.options) = { + scope: FILE, + preserve_unknown_fields: false, + no_default_values_in_constructor: true +}; + +/* ------------------------------------------------------------------ */ +/* GENERIC MATRIX (row-major flat array) */ +/* ------------------------------------------------------------------ */ + +message NumericMatrix { + repeated double values = 1; // length = rows * cols (row-major) + uint32 rows = 2; + uint32 cols = 3; +} + +/* ------------------------------------------------------------------ */ +/* GLOBAL PROFILE (dataset-wide) */ +/* ------------------------------------------------------------------ */ + +message GlobalProfile { + + // ---- basic counts ---- + uint64 samples_used = 1; // how many rows were sampled + uint64 column_count = 2; + uint64 row_count = 3; + + // ---- row null / uniqueness ---- + double row_has_null_ratio = 4; + double row_is_null_ratio = 5; + double unique_row_ratio = 6; + uint64 duplicate_row_count = 7; + + // ---- metadata ---- + string file_type = 8; // “csv”, “parquet”, … + string encoding = 9; // e.g. “utf-8” + + // ---- pairwise stats ---- + NumericMatrix correlation_matrix = 10; + NumericMatrix chi2_matrix = 11; + + // ---- schema map: column-name -> indices (to mirror DataProfiler) --- + map profile_schema = 12; + + // ---- timing ---- + message Times { + double row_stats_ms = 1; + } + Times times = 13; +} + +// helper for profile_schema +message ColumnIndexList { + repeated uint32 indices = 1; +} + +message ColumnStatistics { + + // ---- numeric summaries (nullable when not applicable) ---- + double min = 1; + double max = 2; + double median = 3; + double mean = 4; + double variance = 5; + double stddev = 6; + double skewness = 7; + double kurtosis = 8; + double sum = 9; + + // ---- distribution ---- + repeated double quantiles = 10; // e.g. [q0, q0.5, q1] + uint64 num_zeros = 11; + uint64 num_negatives = 12; + + // ---- uniqueness / cardinality ---- + uint64 unique_count = 13; + double unique_ratio = 14; + + // ---- categorical helpers ---- + bool categorical = 15; + map categorical_count = 16; + + // ---- nulls ---- + uint64 null_count = 17; + repeated string null_types = 18; + + // ---- data-type representation share (DataProfiler style) ---- + map data_type_representation = 19; +} + +/* ------------------------------------------------------------------ */ +/* FULL COLUMN PROFILE */ +/* ------------------------------------------------------------------ */ + +message ColumnProfile { + + // identity + string column_name = 1; + string data_type = 2; // “string”, “int”, “float”, … + string data_label = 3; + // quick hints + bool categorical = 4; + string order = 5; // “random”, “ascending”, “constant value” + + // examples + repeated string samples = 6; // a few raw sample strings + + // heavy stats + ColumnStatistics statistics = 7; +} + +/* ------------------------------------------------------------------ */ +/* TOP-LEVEL CONTAINER */ +/* ------------------------------------------------------------------ */ + +message TableProfile { + GlobalProfile global_profile = 1; + repeated ColumnProfile column_profiles = 2; +} \ No newline at end of file diff --git a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/executionruntimestate.proto b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/executionruntimestate.proto index 3f23c0c44d0..d3328dab318 100644 --- a/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/executionruntimestate.proto +++ b/core/amber/src/main/protobuf/edu/uci/ics/amber/engine/common/executionruntimestate.proto @@ -22,6 +22,7 @@ package edu.uci.ics.amber.engine.common; import "edu/uci/ics/amber/engine/architecture/rpc/controlcommands.proto"; import "edu/uci/ics/amber/engine/architecture/rpc/controlreturns.proto"; import "edu/uci/ics/amber/engine/architecture/worker/statistics.proto"; +import "edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto"; import "edu/uci/ics/amber/core/virtualidentity.proto"; import "edu/uci/ics/amber/core/workflowruntimestate.proto"; import "scalapb/scalapb.proto"; @@ -88,7 +89,8 @@ message ExecutionStatsStore { int64 startTimeStamp = 1; int64 endTimeStamp = 2; map operator_info = 3; - repeated OperatorWorkerMapping operator_worker_mapping = 4; + map operator_table_profile = 4; + repeated OperatorWorkerMapping operator_worker_mapping = 5; } diff --git a/core/amber/src/main/python/core/architecture/handlers/control/query_table_profile_handler.py b/core/amber/src/main/python/core/architecture/handlers/control/query_table_profile_handler.py new file mode 100644 index 00000000000..15c13351ad6 --- /dev/null +++ b/core/amber/src/main/python/core/architecture/handlers/control/query_table_profile_handler.py @@ -0,0 +1,22 @@ +# 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. +from core.architecture.handlers.control.control_handler_base import ControlHandler +from proto.edu.uci.ics.amber.engine.architecture.rpc import (EmptyRequest, TableProfileResponse) + + +class QueryTableProfileHandler(ControlHandler): + async def query_table_profile(self, req: EmptyRequest) -> TableProfileResponse: + return TableProfileResponse(self.context.table_profile_manager.get_table_profile()) \ No newline at end of file diff --git a/core/amber/src/main/python/core/architecture/managers/context.py b/core/amber/src/main/python/core/architecture/managers/context.py index 2da589ada5f..2561f743da3 100644 --- a/core/amber/src/main/python/core/architecture/managers/context.py +++ b/core/amber/src/main/python/core/architecture/managers/context.py @@ -23,6 +23,7 @@ from .debug_manager import DebugManager from .exception_manager import ExceptionManager from .state_processing_manager import StateProcessingManager +from .table_profile_manager import TableProfileManager from .tuple_processing_manager import TupleProcessingManager from .executor_manager import ExecutorManager from .pause_manager import PauseManager @@ -62,6 +63,7 @@ def __init__(self, worker_id, input_queue): ) self.statistics_manager = StatisticsManager() + self.table_profile_manager = TableProfileManager() self.pause_manager = PauseManager( self.input_queue, state_manager=self.state_manager ) diff --git a/core/amber/src/main/python/core/architecture/managers/table_profile_manager.py b/core/amber/src/main/python/core/architecture/managers/table_profile_manager.py new file mode 100644 index 00000000000..715ef48dfce --- /dev/null +++ b/core/amber/src/main/python/core/architecture/managers/table_profile_manager.py @@ -0,0 +1,186 @@ +# 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. + +from core.models import Tuple +from proto.edu.uci.ics.amber.engine.architecture.worker import ( + TableProfile, GlobalProfile, NumericMatrix, ColumnProfile, ColumnIndexList +) +from typing import List, Dict, Any +import pandas as pd +from dataprofiler import Profiler, ProfilerOptions + +# --------------------------------------------------------------------------- # +# Helpers to convert DataProfiler dict ---------------> protobuf TableProfile # +# --------------------------------------------------------------------------- # + +def _to_numeric_matrix(flat: List[float]) -> NumericMatrix: + return NumericMatrix(values=flat, rows=0, cols=0) # TODO: update dimensions + +def _dp_global_to_proto(gjs: Dict[str, Any]) -> GlobalProfile: + gp = GlobalProfile( + samples_used=int(gjs["samples_used"]), + column_count=int(gjs["column_count"]), + row_count=int(gjs["row_count"]), + row_has_null_ratio=gjs["row_has_null_ratio"], + row_is_null_ratio=gjs["row_is_null_ratio"], + unique_row_ratio=gjs["unique_row_ratio"], + duplicate_row_count=int(gjs["duplicate_row_count"]), + file_type=gjs.get("file_type", ""), + encoding=gjs.get("encoding", "") + ) + + correlation = gjs.get("correlation_matrix") + if correlation and isinstance(correlation, list): + flat = [float(x) for row in correlation for x in row] + gp.correlation_matrix = _to_numeric_matrix(flat) + + chi2 = gjs.get("chi2_matrix") + if chi2 and isinstance(chi2, list): + flat = [float(x) for row in chi2 for x in row] + gp.chi2_matrix = _to_numeric_matrix(flat) + + for col, idx_list in gjs.get("profile_schema", {}).items(): + gp.profile_schema[col] = ColumnIndexList(indices=idx_list) + + if "times" in gjs: + gp.times.row_stats_ms = gjs["times"].get("row_stats", 0) * 1_000 + + return gp +def _dp_column_to_proto(cjs: Dict[str, Any]) -> ColumnProfile: + cp = ColumnProfile( + column_name=cjs["column_name"], + data_type=cjs.get("data_type", ""), + data_label=cjs.get("data_label", ""), + categorical=bool(cjs.get("categorical")), + order=cjs.get("order", "") + ) + samples = cjs.get("samples", []) + if isinstance(samples, list): + cp.samples.extend([str(s) for s in samples[:10]]) + elif isinstance(samples, str): + # compact-format DP report returns a JSON list-as-string + sample_str = samples.strip("[]") + cp.samples.extend([s.strip(" '") for s in sample_str.split(",")[:10]]) + + stats = cjs.get("statistics", {}) + cs = cp.statistics + + numeric_fields = ("min", "max", "median", "mean", "variance", "stddev", "skewness", "kurtosis", "sum") + for field in numeric_fields: + value = stats.get(field) + if value not in (None, "nan"): + try: + # only convert if it looks like a number + cs_field = float(value) + setattr(cs, field, cs_field) + except (ValueError, TypeError): + # skip non-floatable values like datetime strings + pass + + if "quantiles" in stats: + try: + cs.quantiles.extend([float(v) for _, v in sorted(stats["quantiles"].items())]) + except Exception: + pass + + cs.num_zeros = int(stats.get("num_zeros", 0)) + cs.num_negatives = int(stats.get("num_negatives", 0)) + cs.unique_count = int(stats.get("unique_count", 0)) + cs.unique_ratio = float(stats.get("unique_ratio", 0)) + cs.null_count = int(stats.get("null_count", 0)) + cs.null_types.extend(stats.get("null_types", [])) + + if "categorical_count" in stats: + for cat, cnt in stats["categorical_count"].items(): + cp.statistics.categorical_count[cat] = int(cnt) + + dtr = stats.get("data_type_representation", {}) + for k, v in dtr.items(): + try: + cp.statistics.data_type_representation[k] = float(v) + except Exception: + pass + + return cp + +def dp_report_to_tableprofile(report: Dict[str, Any]) -> TableProfile: + tp = TableProfile() + tp.global_profile = _dp_global_to_proto(report["global_stats"]) + for col_js in report["data_stats"]: + tp.column_profiles.append(_dp_column_to_proto(col_js)) + return tp + +# --------------------------------------------------------------------------- # +# TableProfileManager (Python version) # +# --------------------------------------------------------------------------- # + +class TableProfileManager: + def __init__(self): + self._rows: List[Dict[str, Any]] = [] + self._profile_proto: TableProfile | None = None + self._dirty: bool = False # ← tracks if buffer changed + + profiler_options = ProfilerOptions() + profiler_options.set({ + "structured_options.data_labeler.is_enabled": True + }) + self.profiler_options = profiler_options + + def update_table_profile(self, tup: Tuple): + row_dict = tup.as_dict() + self._rows.append(row_dict) + self._dirty = True + + def get_table_profile(self) -> TableProfile: + """ + Return the protobuf `TableProfile`. + + It is rebuilt iff (a) we do not have one yet, or (b) new rows were + appended since the last build (`_dirty` is True). + """ + if self._profile_proto is None or self._dirty: + self._profile_proto = self._build_profile() + return self._profile_proto + + def _build_profile(self) -> TableProfile: + """ + Build (or rebuild) the profile from the current buffer. + Called only by `get_table_profile`. + """ + try: + if not self._rows: + # no data yet – return an empty proto + return TableProfile() + + df = pd.DataFrame(self._rows) + profile = Profiler( + df, + options=self.profiler_options, + profiler_type="structured", + ) + report = profile.report(report_options={"output_format": "compact"}) + return dp_report_to_tableprofile(report) + finally: + # whether succeeded or failed, mark buffer as clean so that we + # don’t rebuild until new tuples arrive + self._dirty = False + + def to_bytes(self) -> bytes: + return self.get_table_profile().SerializeToString() + + def reset(self): + self._rows.clear() + self._profile_proto = None diff --git a/core/amber/src/main/python/core/architecture/managers/test_table_profile_manager.py b/core/amber/src/main/python/core/architecture/managers/test_table_profile_manager.py new file mode 100644 index 00000000000..a779caa22b0 --- /dev/null +++ b/core/amber/src/main/python/core/architecture/managers/test_table_profile_manager.py @@ -0,0 +1,146 @@ +# 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 datetime +import pytest +from core.models import Tuple +from core.models.schema.schema import Schema +from core.architecture.managers.table_profile_manager import TableProfileManager +from proto.edu.uci.ics.amber.engine.architecture.worker import TableProfile + + +@pytest.fixture +def complex_schema() -> Schema: + return Schema( + raw_schema={ + "id": "INTEGER", + "name": "STRING", + "email": "STRING", + "signup_date": "TIMESTAMP", + "age": "INTEGER", + "height_cm": "DOUBLE", + "is_active": "BOOLEAN", + "phone": "STRING", + "zip_code": "STRING", + } + ) + + +@pytest.fixture +def sample_rows(complex_schema) -> list[Tuple]: + return [ + Tuple( + { + "id": 1, + "name": "Alice", + "email": "alice@example.com", + "signup_date": datetime.datetime(2023, 1, 15, 9, 0), + "age": 30, + "height_cm": 165.4, + "is_active": True, + "phone": "555-1234", + "zip_code": "90001" + }, + schema=complex_schema, + ), + Tuple( + { + "id": 2, + "name": "Bob", + "email": "bob@example.com", + "signup_date": datetime.datetime(2022, 5, 20, 14, 30), + "age": 42, + "height_cm": 178.2, + "is_active": False, + "phone": "555-5678", + "zip_code": "10001" + }, + schema=complex_schema, + ), + Tuple( + { + "id": 3, + "name": "Charlie", + "email": "charlie@example.com", + "signup_date": datetime.datetime(2021, 9, 10, 8, 45), + "age": 28, + "height_cm": 172.0, + "is_active": True, + "phone": "555-9999", + "zip_code": "60601" + }, + schema=complex_schema, + ), + Tuple( + { + "id": 4, + "name": "Diana", + "email": "diana@example.net", + "signup_date": datetime.datetime(2023, 2, 25, 17, 15), + "age": 35, + "height_cm": 160.0, + "is_active": True, + "phone": "555-1212", + "zip_code": "94105" + }, + schema=complex_schema, + ), + Tuple( + { + "id": 5, + "name": "Ethan", + "email": "ethan@example.org", + "signup_date": datetime.datetime(2024, 4, 2, 11, 5), + "age": 25, + "height_cm": 180.7, + "is_active": False, + "phone": "555-7777", + "zip_code": "30301" + }, + schema=complex_schema, + ), + ] + + +@pytest.fixture +def mgr(sample_rows) -> TableProfileManager: + mgr = TableProfileManager() + for t in sample_rows: + mgr.update_table_profile(t) + return mgr + + +class TestTableProfileManager: + def test_it_builds_valid_profile(self, mgr: TableProfileManager): + profile: TableProfile = mgr.get_table_profile() + + # Validate general structure + assert profile.global_profile.samples_used > 0 + assert profile.global_profile.column_count >= 9 + assert profile.global_profile.row_count >= 5 + + assert len(profile.column_profiles) == profile.global_profile.column_count + + # Check that known columns have expected labels/fields populated + col_names = [c.column_name for c in profile.column_profiles] + for name in ["email", "signup_date", "phone", "zip_code"]: + assert name in col_names + + # Check statistics were collected for a sample column + email_col = next((c for c in profile.column_profiles if c.column_name == "email"), None) + assert email_col is not None + assert email_col.statistics.unique_count > 0 + assert email_col.data_type == "string" \ No newline at end of file diff --git a/core/amber/src/main/python/core/architecture/rpc/async_rpc_handler_initializer.py b/core/amber/src/main/python/core/architecture/rpc/async_rpc_handler_initializer.py index 5d846c19929..d17ff02d28c 100644 --- a/core/amber/src/main/python/core/architecture/rpc/async_rpc_handler_initializer.py +++ b/core/amber/src/main/python/core/architecture/rpc/async_rpc_handler_initializer.py @@ -39,6 +39,7 @@ from core.architecture.handlers.control.query_statistics_handler import ( QueryStatisticsHandler, ) +from core.architecture.handlers.control.query_table_profile_handler import QueryTableProfileHandler from core.architecture.handlers.control.replay_current_tuple_handler import ( RetryCurrentTupleHandler, ) @@ -56,6 +57,7 @@ class AsyncRPCHandlerInitializer( OpenExecutorHandler, PauseWorkerHandler, QueryStatisticsHandler, + QueryTableProfileHandler, RetryCurrentTupleHandler, ResumeWorkerHandler, StartWorkerHandler, diff --git a/core/amber/src/main/python/core/runnables/main_loop.py b/core/amber/src/main/python/core/runnables/main_loop.py index 5ec02e81f56..a90736468fe 100644 --- a/core/amber/src/main/python/core/runnables/main_loop.py +++ b/core/amber/src/main/python/core/runnables/main_loop.py @@ -165,6 +165,7 @@ def process_input_tuple(self) -> None: for output_tuple in self.process_tuple_with_udf(): self._check_and_process_control() if output_tuple is not None: + self.context.table_profile_manager.update_table_profile(output_tuple) self.context.statistics_manager.increase_output_statistics( PortIdentity(0), output_tuple.in_mem_size() ) diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/core/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/core/__init__.py index 1e57298fe5f..acf6f6355aa 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/core/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/core/__init__.py @@ -22,7 +22,9 @@ from dataclasses import dataclass from datetime import datetime -from typing import List +from typing import ( + List, +) import betterproto diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py index ae4a12611b1..cb20f34d898 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/rpc/__init__.py @@ -435,6 +435,9 @@ class ControlReturn(betterproto.Message): finalize_checkpoint_response: "FinalizeCheckpointResponse" = ( betterproto.message_field(52, group="sealed_value") ) + table_profile_response: "TableProfileResponse" = betterproto.message_field( + 53, group="sealed_value" + ) control_error: "ControlError" = betterproto.message_field(101, group="sealed_value") """common responses""" diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/sendsemantics/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/sendsemantics/__init__.py index 0b10145b8a2..14be5143a07 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/sendsemantics/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/sendsemantics/__init__.py @@ -21,7 +21,9 @@ # This file has been @generated from dataclasses import dataclass -from typing import List +from typing import ( + List, +) import betterproto diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py index f5fd615493d..ec387b457f5 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py @@ -16,12 +16,15 @@ # under the License. # Generated by the protocol buffer compiler. DO NOT EDIT! -# sources: edu/uci/ics/amber/engine/architecture/worker/statistics.proto +# sources: edu/uci/ics/amber/engine/architecture/worker/statistics.proto, edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto # plugin: python-betterproto # This file has been @generated from dataclasses import dataclass -from typing import List +from typing import ( + Dict, + List, +) import betterproto @@ -61,3 +64,115 @@ class WorkerStatistics(betterproto.Message): class WorkerMetrics(betterproto.Message): worker_state: "WorkerState" = betterproto.enum_field(1) worker_statistics: "WorkerStatistics" = betterproto.message_field(2) + + +@dataclass(eq=False, repr=False) +class NumericMatrix(betterproto.Message): + values: List[float] = betterproto.double_field(1) + rows: int = betterproto.uint32_field(2) + cols: int = betterproto.uint32_field(3) + + +@dataclass(eq=False, repr=False) +class GlobalProfile(betterproto.Message): + samples_used: int = betterproto.uint64_field(1) + """---- basic counts ----""" + + column_count: int = betterproto.uint64_field(2) + row_count: int = betterproto.uint64_field(3) + row_has_null_ratio: float = betterproto.double_field(4) + """---- row null / uniqueness ----""" + + row_is_null_ratio: float = betterproto.double_field(5) + unique_row_ratio: float = betterproto.double_field(6) + duplicate_row_count: int = betterproto.uint64_field(7) + file_type: str = betterproto.string_field(8) + """---- metadata ----""" + + encoding: str = betterproto.string_field(9) + correlation_matrix: "NumericMatrix" = betterproto.message_field(10) + """---- pairwise stats ----""" + + chi2_matrix: "NumericMatrix" = betterproto.message_field(11) + profile_schema: Dict[str, "ColumnIndexList"] = betterproto.map_field( + 12, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE + ) + """---- schema map: column-name -> indices (to mirror DataProfiler) ---""" + + times: "GlobalProfileTimes" = betterproto.message_field(13) + + +@dataclass(eq=False, repr=False) +class GlobalProfileTimes(betterproto.Message): + """---- timing ----""" + + row_stats_ms: float = betterproto.double_field(1) + + +@dataclass(eq=False, repr=False) +class ColumnIndexList(betterproto.Message): + """helper for profile_schema""" + + indices: List[int] = betterproto.uint32_field(1) + + +@dataclass(eq=False, repr=False) +class ColumnStatistics(betterproto.Message): + min: float = betterproto.double_field(1) + """---- numeric summaries (nullable when not applicable) ----""" + + max: float = betterproto.double_field(2) + median: float = betterproto.double_field(3) + mean: float = betterproto.double_field(4) + variance: float = betterproto.double_field(5) + stddev: float = betterproto.double_field(6) + skewness: float = betterproto.double_field(7) + kurtosis: float = betterproto.double_field(8) + sum: float = betterproto.double_field(9) + quantiles: List[float] = betterproto.double_field(10) + """---- distribution ----""" + + num_zeros: int = betterproto.uint64_field(11) + num_negatives: int = betterproto.uint64_field(12) + unique_count: int = betterproto.uint64_field(13) + """---- uniqueness / cardinality ----""" + + unique_ratio: float = betterproto.double_field(14) + categorical: bool = betterproto.bool_field(15) + """---- categorical helpers ----""" + + categorical_count: Dict[str, int] = betterproto.map_field( + 16, betterproto.TYPE_STRING, betterproto.TYPE_UINT64 + ) + null_count: int = betterproto.uint64_field(17) + """---- nulls ----""" + + null_types: List[str] = betterproto.string_field(18) + data_type_representation: Dict[str, float] = betterproto.map_field( + 19, betterproto.TYPE_STRING, betterproto.TYPE_DOUBLE + ) + """---- data-type representation share (DataProfiler style) ----""" + + +@dataclass(eq=False, repr=False) +class ColumnProfile(betterproto.Message): + column_name: str = betterproto.string_field(1) + """identity""" + + data_type: str = betterproto.string_field(2) + data_label: str = betterproto.string_field(3) + categorical: bool = betterproto.bool_field(4) + """quick hints""" + + order: str = betterproto.string_field(5) + samples: List[str] = betterproto.string_field(6) + """examples""" + + statistics: "ColumnStatistics" = betterproto.message_field(7) + """heavy stats""" + + +@dataclass(eq=False, repr=False) +class TableProfile(betterproto.Message): + global_profile: "GlobalProfile" = betterproto.message_field(1) + column_profiles: List["ColumnProfile"] = betterproto.message_field(2) diff --git a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py index 1a0886a7c5b..265636ac02c 100644 --- a/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py +++ b/core/amber/src/main/python/proto/edu/uci/ics/amber/engine/common/__init__.py @@ -160,8 +160,11 @@ class ExecutionStatsStore(betterproto.Message): operator_info: Dict[str, "OperatorMetrics"] = betterproto.map_field( 3, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE ) + operator_table_profile: Dict[str, "_architecture_worker__.TableProfile"] = ( + betterproto.map_field(4, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE) + ) operator_worker_mapping: List["OperatorWorkerMapping"] = betterproto.message_field( - 4 + 5 ) diff --git a/core/amber/src/main/python/pytexera/storage/dataset_file_document.py b/core/amber/src/main/python/pytexera/storage/dataset_file_document.py index 28ffe0d30e5..2bf01f58b88 100644 --- a/core/amber/src/main/python/pytexera/storage/dataset_file_document.py +++ b/core/amber/src/main/python/pytexera/storage/dataset_file_document.py @@ -19,7 +19,14 @@ import io import requests import urllib.parse +import pandas as pd +from core.models import TableLike, Table +from pandas.api.types import ( + is_float_dtype, + is_integer_dtype, + is_datetime64_any_dtype, +) class DatasetFileDocument: def __init__(self, file_path: str): @@ -96,3 +103,80 @@ def read_file(self) -> io.BytesIO: ) return io.BytesIO(response.content) + + def read_as_table(self, schema: dict[str, str] | None = None, **pandas_kwargs) -> "TableLike": + """ + Download the file and materialise it as a pandas DataFrame. + + Parameters + ---------- + **pandas_kwargs : + Extra keyword arguments forwarded to the relevant + ``pandas.read_*`` function (e.g., ``read_csv``). + + Returns + ------- + TableLike (currently a pandas.DataFrame) + The tabular representation of the file’s contents. + + Notes + ----- + This is a *hacky* helper—intended only for local Python-side + experimentation. For production use, push the logic into a + proper service layer. + """ + + # Pull the bytes from object storage + # Pull the bytes from object storage + file_bytes = self.read_file() + + # Infer file format from the extension + ext = self.file_relative_path.rsplit(".", 1)[-1].lower() + + # ---- 1) load the file ----------------------------------------- + if ext in {"csv", "tsv", "txt"}: + # default separator if caller didn't pass one + pandas_kwargs.setdefault("sep", "," if ext == "csv" else "\t") + # keep blank rows if they exist; treat empty cells as NA + pandas_kwargs.setdefault("skip_blank_lines", False) + pandas_kwargs.setdefault("keep_default_na", True) + df = pd.read_csv(file_bytes, **pandas_kwargs) + + elif ext in {"json", "ndjson"}: + df = pd.read_json(file_bytes, lines=(ext == "ndjson"), **pandas_kwargs) + elif ext == "parquet": + df = pd.read_parquet(file_bytes, **pandas_kwargs) + else: + raise ValueError(f"Unsupported file type: .{ext}") + + # ---- 2) hard-cast columns according to Amber schema ----------- + if schema: + for col, amber_type in schema.items(): + if col not in df.columns: + continue + + s = df[col] # shorthand + + if amber_type in {"INTEGER", "LONG"}: + df[col] = pd.to_numeric(s, errors="coerce").astype("Int64") + + elif amber_type == "DOUBLE": + df[col] = pd.to_numeric(s, errors="coerce").astype("float64") + + elif amber_type == "BOOLEAN": + df[col] = s.astype("boolean") # nullable boolean + + elif amber_type == "STRING": + # nullable *string* dtype – keeps pd.NA for empty cells + df[col] = s.astype(pd.StringDtype()) + + elif amber_type == "TIMESTAMP": + df[col] = ( + pd.to_datetime(s, errors="coerce") + .dt.tz_localize(None) + .dt.to_pydatetime() + ) + else: # BINARY, ANY, unknown + raise Exception(f"Unsupported type: {amber_type}") + + return Table(df) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala index bcee1b896e2..4a2dbd12a1d 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/common/AmberProcessor.scala @@ -27,7 +27,10 @@ import edu.uci.ics.amber.engine.architecture.messaginglayer.{ import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.ControlInvocation import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.ReturnInvocation import edu.uci.ics.amber.engine.architecture.worker.WorkflowWorker.MainThreadDelegateMessage -import edu.uci.ics.amber.engine.architecture.worker.managers.StatisticsManager +import edu.uci.ics.amber.engine.architecture.worker.managers.{ + StatisticsManager, + TableProfileManager +} import edu.uci.ics.amber.engine.common.AmberLogging import edu.uci.ics.amber.engine.common.ambermessage.{ DirectControlMessagePayload, @@ -61,6 +64,8 @@ abstract class AmberProcessor( // statistics manager val statisticsManager: StatisticsManager = new StatisticsManager() + // table profile manager + val tableProfileManager: TableProfileManager = new TableProfileManager() def processDCM( channelId: ChannelIdentity, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala index d9cfe9c6893..775ecf93a00 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/ClientEvent.scala @@ -24,12 +24,16 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregat import edu.uci.ics.amber.engine.common.ambermessage.WorkflowFIFOMessagePayload import edu.uci.ics.amber.engine.common.executionruntimestate.OperatorMetrics import edu.uci.ics.amber.core.virtualidentity.ActorVirtualIdentity +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile trait ClientEvent extends WorkflowFIFOMessagePayload case class ExecutionStateUpdate(state: WorkflowAggregatedState) extends ClientEvent -case class ExecutionStatsUpdate(operatorMetrics: Map[String, OperatorMetrics]) extends ClientEvent +case class ExecutionStatsUpdate( + operatorMetrics: Map[String, OperatorMetrics], + operatorTableProfiles: Map[String, TableProfile] +) extends ClientEvent case class ReportCurrentProcessingTuple( operatorID: String, diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala index bd54fd2613e..fdf6319a5e7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/Controller.scala @@ -229,7 +229,8 @@ class Controller( outputMessages.foreach(transferService.send) cp.asyncRPCClient.sendToClient( ExecutionStatsUpdate( - cp.workflowExecution.getAllRegionExecutionsStats + cp.workflowExecution.getAllRegionExecutionsStats, + cp.workflowExecution.getAllRegionExecutionTableProfiles ) ) globalReplayManager.markRecoveryStatus(CONTROLLER, isRecovering = false) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala index b6f1b93fdcd..7cbab425c66 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/OperatorExecution.scala @@ -29,6 +29,7 @@ import edu.uci.ics.amber.engine.architecture.worker.statistics.{ import edu.uci.ics.amber.engine.common.executionruntimestate.{OperatorMetrics, OperatorStatistics} import edu.uci.ics.amber.core.virtualidentity.ActorVirtualIdentity import edu.uci.ics.amber.core.workflow.PortIdentity +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile import java.util import scala.jdk.CollectionConverters._ @@ -102,6 +103,13 @@ case class OperatorExecution() { ) } + def getTableProfile: TableProfile = { + val workerTableProfiles = workerExecutions.values.asScala.map(_.getTableProfile) + // TODO: by default, choose the first worker's table profile + // this option will fail when there are multiple workers + workerTableProfiles.head + } + def isInputPortCompleted(portId: PortIdentity): Boolean = { workerExecutions .values() diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala index e78bac29567..f69510220a6 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/RegionExecution.scala @@ -26,6 +26,7 @@ import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerStatistics import edu.uci.ics.amber.engine.common.executionruntimestate.OperatorMetrics import edu.uci.ics.amber.core.virtualidentity.PhysicalOpIdentity import edu.uci.ics.amber.core.workflow.PhysicalLink +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile import scala.collection.mutable @@ -117,6 +118,13 @@ case class RegionExecution(region: Region) { }.toMap } + def getTableProfiles: Map[PhysicalOpIdentity, TableProfile] = { + operatorExecutions.map { + case (physicalOpId, operatorExecution) => + physicalOpId -> operatorExecution.getTableProfile + }.toMap + } + def isCompleted: Boolean = getState == WorkflowAggregatedState.COMPLETED def getState: WorkflowAggregatedState = { diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala index b36e8b27b4b..6ca68550f9b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/execution/WorkflowExecution.scala @@ -25,6 +25,7 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.WorkflowAggregat import edu.uci.ics.amber.engine.architecture.scheduling.{Region, RegionIdentity} import edu.uci.ics.amber.engine.common.executionruntimestate.OperatorMetrics import edu.uci.ics.amber.core.virtualidentity.PhysicalOpIdentity +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile import scala.collection.mutable @@ -96,6 +97,27 @@ case class WorkflowExecution() { aggregatedStats } + /** + * Retrieve the runtime table profiles of all `RegionExecutions` + * + * @return A `Map` with key being `Logical Operator ID` and the value being operator table profile + */ + def getAllRegionExecutionTableProfiles: Map[String, TableProfile] = { + val allRegionExecutions: Iterable[RegionExecution] = getAllRegionExecutions + val tableProfileMap: Map[PhysicalOpIdentity, TableProfile] = allRegionExecutions.flatMap { + regionExecution => + regionExecution.getTableProfiles.map { + case (physicalOpIdentity, tableProfile) => + (physicalOpIdentity, tableProfile) + } + }.toMap + + tableProfileMap.groupBy(_._1.logicalOpId.id).map { + case (logicalOpId, tableProfiles) => + (logicalOpId, tableProfiles.values.head) + } + } + /** * Retrieves all `RegionExecutions`, preserving the order in which they were created. * diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala index 11c4a38ff1d..6b22c024362 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/PauseHandler.scala @@ -78,7 +78,8 @@ trait PauseHandler { // update frontend workflow status sendToClient( ExecutionStatsUpdate( - cp.workflowExecution.getAllRegionExecutionsStats + cp.workflowExecution.getAllRegionExecutionsStats, + cp.workflowExecution.getAllRegionExecutionTableProfiles ) ) sendToClient(ExecutionStateUpdate(cp.workflowExecution.getState)) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala index 9f2782489cb..97867066f67 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/QueryWorkerStatisticsHandler.scala @@ -19,7 +19,8 @@ package edu.uci.ics.amber.engine.architecture.controller.promisehandlers -import com.twitter.util.Future +import com.twitter.util.{Future, Promise} +import edu.uci.ics.amber.core.virtualidentity.ActorVirtualIdentity import edu.uci.ics.amber.engine.architecture.controller.{ ControllerAsyncRPCHandlerInitializer, ExecutionStatsUpdate @@ -30,58 +31,62 @@ import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{ QueryStatisticsRequest } import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.EmptyReturn +import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile import edu.uci.ics.amber.util.VirtualIdentityUtils -/** Get statistics from all the workers - * - * possible sender: controller(by statusUpdateAskHandle) - */ -trait QueryWorkerStatisticsHandler { - this: ControllerAsyncRPCHandlerInitializer => +trait QueryWorkerStatisticsHandler { this: ControllerAsyncRPCHandlerInitializer => override def controllerInitiateQueryStatistics( msg: QueryStatisticsRequest, ctx: AsyncRPCContext ): Future[EmptyReturn] = { - // send to specified workers (or all workers by default) - val workers = if (msg.filterByWorkers.nonEmpty) { - msg.filterByWorkers - } else { - cp.workflowExecution.getAllRegionExecutions - .flatMap(_.getAllOperatorExecutions.map(_._2)) - .flatMap(_.getWorkerIds) - } - // send QueryStatistics message - val requests = workers - .map(workerId => - // must immediately update worker state and stats after reply - workerInterface - .queryStatistics(EmptyRequest(), workerId) - .map(resp => { - val workerExecution = - cp.workflowExecution - .getLatestOperatorExecution(VirtualIdentityUtils.getPhysicalOpId(workerId)) - .getWorkerExecution(workerId) - workerExecution.setState(resp.metrics.workerState) - workerExecution.setStats(resp.metrics.workerStatistics) - }) - ) - .toSeq + // 1. decide whom to contact + val workers: Iterable[ActorVirtualIdentity] = + if (msg.filterByWorkers.nonEmpty) msg.filterByWorkers + else + cp.workflowExecution.getAllRegionExecutions + .flatMap(_.getAllOperatorExecutions.map(_._2)) + .flatMap(_.getWorkerIds) + + // 2. fire queries in parallel + val requests: Seq[Future[Unit]] = workers.map { wid => + val exec = cp.workflowExecution + .getLatestOperatorExecution(VirtualIdentityUtils.getPhysicalOpId(wid)) + .getWorkerExecution(wid) + + // query metrics first + workerInterface.queryStatistics(EmptyRequest(), wid).flatMap { stat => + // update state & basic stats immediately + exec.setState(stat.metrics.workerState) + exec.setStats(stat.metrics.workerStatistics) - // wait for all workers to reply before notifying frontend - Future - .collect(requests) - .map(_ => - sendToClient( - ExecutionStatsUpdate( - cp.workflowExecution.getAllRegionExecutionsStats - ) + if (stat.metrics.workerState == WorkerState.COMPLETED) { + // worker finished – fetch its profile too + workerInterface + .queryTableProfile(EmptyRequest(), wid) + .map { prof => + exec.setTableProfile(prof.tableProfiles) + } + } else { + // not completed – record an empty profile + exec.setTableProfile(new TableProfile(None, Seq.empty)) + Future.Unit + } + } + }.toSeq + + // 3. when all workers replied, push aggregated view to UI + Future.collect(requests).map { _ => + sendToClient( + ExecutionStatsUpdate( + cp.workflowExecution.getAllRegionExecutionsStats, + cp.workflowExecution.getAllRegionExecutionTableProfiles ) ) .map { _ => EmptyReturn() } } - } diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala index f225d991038..2c99060cd7e 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/ResumeHandler.scala @@ -57,7 +57,8 @@ trait ResumeHandler { // update frontend status sendToClient( ExecutionStatsUpdate( - cp.workflowExecution.getAllRegionExecutionsStats + cp.workflowExecution.getAllRegionExecutionsStats, + cp.workflowExecution.getAllRegionExecutionTableProfiles ) ) cp.controllerTimerService diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala index 4330e4c851b..70a165ab7d5 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/controller/promisehandlers/WorkerStateUpdatedHandler.scala @@ -52,7 +52,8 @@ trait WorkerStateUpdatedHandler { ) sendToClient( ExecutionStatsUpdate( - cp.workflowExecution.getAllRegionExecutionsStats + cp.workflowExecution.getAllRegionExecutionsStats, + cp.workflowExecution.getAllRegionExecutionTableProfiles ) ) EmptyReturn() diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala index b4e37eeb49e..ecf06bf8eb4 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/deploysemantics/layer/WorkerExecution.scala @@ -23,6 +23,7 @@ import edu.uci.ics.amber.engine.architecture.controller.execution.WorkerPortExec import edu.uci.ics.amber.engine.architecture.worker.statistics.WorkerState.UNINITIALIZED import edu.uci.ics.amber.engine.architecture.worker.statistics.{WorkerState, WorkerStatistics} import edu.uci.ics.amber.core.workflow.PortIdentity +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile import scala.collection.mutable @@ -36,6 +37,7 @@ case class WorkerExecution() extends Serializable { private var state: WorkerState = UNINITIALIZED private var stats: WorkerStatistics = WorkerStatistics(Seq.empty, Seq.empty, 0, 0, 0) + private var tableProfile: TableProfile = TableProfile(None, Seq.empty) def getState: WorkerState = state @@ -49,6 +51,12 @@ case class WorkerExecution() extends Serializable { this.stats = stats } + def getTableProfile: TableProfile = tableProfile + + def setTableProfile(tableProfile: TableProfile): Unit = { + this.tableProfile = tableProfile + } + def getInputPortExecution(portId: PortIdentity): WorkerPortExecution = { if (!inputPortExecutions.contains(portId)) { inputPortExecutions(portId) = new WorkerPortExecution() diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 77d35012346..c47aac67ef9 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -197,7 +197,10 @@ class RegionExecutionCoordinator( val regionExecution = workflowExecution.getRegionExecution(region.id) asyncRPCClient.sendToClient( - ExecutionStatsUpdate(workflowExecution.getAllRegionExecutionsStats) + ExecutionStatsUpdate( + workflowExecution.getAllRegionExecutionsStats, + workflowExecution.getAllRegionExecutionTableProfiles + ) ) asyncRPCClient.sendToClient( WorkerAssignmentUpdate( @@ -412,7 +415,8 @@ class RegionExecutionCoordinator( ): Future[Seq[Unit]] = { asyncRPCClient.sendToClient( ExecutionStatsUpdate( - workflowExecution.getAllRegionExecutionsStats + workflowExecution.getAllRegionExecutionsStats, + workflowExecution.getAllRegionExecutionTableProfiles ) ) val allStarterOperators = region.getStarterOperators diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala index e4674f7a339..644e74608ed 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessor.scala @@ -119,7 +119,7 @@ class DataProcessor( ) statisticsManager.increaseInputStatistics(portIdentity, tuple.inMemSize) - + tableProfileManager.updateTableProfile(tuple) } catch safely { case e => // forward input tuple to the user and pause DP thread diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala index 442372a482a..b321889d8ec 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/DataProcessorRPCHandlerInitializer.scala @@ -42,6 +42,7 @@ class DataProcessorRPCHandlerInitializer(val dp: DataProcessor) with PauseHandler with AddPartitioningHandler with QueryStatisticsHandler + with QueryTableProfileHandler with ResumeHandler with StartHandler with StartChannelHandler diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/TableProfileManager.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/TableProfileManager.scala new file mode 100644 index 00000000000..1c6edb012e3 --- /dev/null +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/managers/TableProfileManager.scala @@ -0,0 +1,28 @@ +/* + * 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. + */ + +package edu.uci.ics.amber.engine.architecture.worker.managers + +import edu.uci.ics.amber.core.tuple.Tuple +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile + +class TableProfileManager { + def updateTableProfile(tuple: Tuple): Unit = {} + + def getTableProfile: TableProfile = TableProfile(globalProfile = None, columnProfiles = List()) +} diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryTableProfileHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryTableProfileHandler.scala new file mode 100644 index 00000000000..b84da721332 --- /dev/null +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/QueryTableProfileHandler.scala @@ -0,0 +1,35 @@ +/* + * 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. + */ + +package edu.uci.ics.amber.engine.architecture.worker.promisehandlers + +import com.twitter.util.Future +import edu.uci.ics.amber.engine.architecture.rpc.controlcommands.{AsyncRPCContext, EmptyRequest} +import edu.uci.ics.amber.engine.architecture.rpc.controlreturns.TableProfileResponse +import edu.uci.ics.amber.engine.architecture.worker.DataProcessorRPCHandlerInitializer + +trait QueryTableProfileHandler { + this: DataProcessorRPCHandlerInitializer => + + override def queryTableProfile( + request: EmptyRequest, + ctx: AsyncRPCContext + ): Future[TableProfileResponse] = { + TableProfileResponse(dp.tableProfileManager.getTableProfile) + } +} diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/OperatorStatisticsUpdateEvent.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/OperatorStatisticsUpdateEvent.scala index 3e66da4b946..93837975bf7 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/OperatorStatisticsUpdateEvent.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/model/websocket/event/OperatorStatisticsUpdateEvent.scala @@ -18,6 +18,7 @@ */ package edu.uci.ics.texera.web.model.websocket.event +import edu.uci.ics.amber.engine.architecture.worker.tableprofile.TableProfile case class OperatorAggregatedMetrics( operatorState: String, @@ -31,5 +32,7 @@ case class OperatorAggregatedMetrics( aggregatedIdleTime: Long ) -case class OperatorStatisticsUpdateEvent(operatorStatistics: Map[String, OperatorAggregatedMetrics]) - extends TexeraWebSocketEvent +case class OperatorStatisticsUpdateEvent( + operatorStatistics: Map[String, OperatorAggregatedMetrics], + operatorResultTableProfiles: Map[String, TableProfile] +) extends TexeraWebSocketEvent diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala index 8e73bfd32eb..40735c88543 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/resource/WorkflowWebsocketResource.scala @@ -49,8 +49,19 @@ class WorkflowWebsocketResource extends LazyLogging { def myOnOpen(session: Session, config: EndpointConfig): Unit = { val sessionState = new SessionState(session) SessionState.setState(session.getId, sessionState) - val wid = session.getRequestParameterMap.get("wid").get(0).toLong - val cuid = session.getRequestParameterMap.get("cuid").get(0).toInt + val paramMap = session.getRequestParameterMap + + val wid = paramMap.get("wid").get(0).toLong + val cuid = paramMap.get("cuid").get(0).toInt + // Hacky solution for local Python source purpose: set the JWT token from request into a system property + if (paramMap.containsKey("access-token")) { + val tokenList = paramMap.get("access-token") + if (tokenList != null && !tokenList.isEmpty) { + val token = tokenList.get(0) + System.setProperty("USER_JWT_TOKEN", token) + } + } + // hack to refresh frontend run button state sessionState.send(WorkflowStateEvent("Uninitialized")) val workflowState = diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala index 05523ea2afb..6fea1e6b4cb 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/service/ExecutionStatsService.scala @@ -102,24 +102,29 @@ class ExecutionStatsService( addSubscription( stateStore.statsStore.registerDiffHandler((oldState, newState) => { // Update operator stats if any operator updates its stat - if (newState.operatorInfo.toSet != oldState.operatorInfo.toSet) { + if ( + newState.operatorInfo.toSet != oldState.operatorInfo.toSet || newState.operatorTableProfile != oldState.operatorTableProfile + ) { Iterable( - OperatorStatisticsUpdateEvent(newState.operatorInfo.collect { - case x => - val metrics = x._2 - val res = OperatorAggregatedMetrics( - Utils.aggregatedStateToString(metrics.operatorState), - metrics.operatorStatistics.inputMetrics.map(_.tupleMetrics.count).sum, - metrics.operatorStatistics.inputMetrics.map(_.tupleMetrics.size).sum, - metrics.operatorStatistics.outputMetrics.map(_.tupleMetrics.count).sum, - metrics.operatorStatistics.outputMetrics.map(_.tupleMetrics.size).sum, - metrics.operatorStatistics.numWorkers, - metrics.operatorStatistics.dataProcessingTime, - metrics.operatorStatistics.controlProcessingTime, - metrics.operatorStatistics.idleTime - ) - (x._1, res) - }) + OperatorStatisticsUpdateEvent( + newState.operatorInfo.collect { + case x => + val metrics = x._2 + val res = OperatorAggregatedMetrics( + Utils.aggregatedStateToString(metrics.operatorState), + metrics.operatorStatistics.inputMetrics.map(_.tupleMetrics.count).sum, + metrics.operatorStatistics.inputMetrics.map(_.tupleMetrics.size).sum, + metrics.operatorStatistics.outputMetrics.map(_.tupleMetrics.count).sum, + metrics.operatorStatistics.outputMetrics.map(_.tupleMetrics.size).sum, + metrics.operatorStatistics.numWorkers, + metrics.operatorStatistics.dataProcessingTime, + metrics.operatorStatistics.controlProcessingTime, + metrics.operatorStatistics.idleTime + ) + (x._1, res) + }, + newState.operatorTableProfile + ) ) } else { Iterable.empty @@ -178,7 +183,9 @@ class ExecutionStatsService( client .registerCallback[ExecutionStatsUpdate]((evt: ExecutionStatsUpdate) => { stateStore.statsStore.updateState { statsStore => - statsStore.withOperatorInfo(evt.operatorMetrics) + statsStore + .withOperatorInfo(evt.operatorMetrics) + .withOperatorTableProfile(evt.operatorTableProfiles) } metricsPersistThread.foreach { thread => thread.execute(() => { diff --git a/core/gui/package.json b/core/gui/package.json index 20b02f092b6..aa9c4be04a9 100644 --- a/core/gui/package.json +++ b/core/gui/package.json @@ -43,6 +43,7 @@ "@ngx-formly/core": "6.3.0", "@ngx-formly/ng-zorro-antd": "6.3.0", "@stoplight/json-ref-resolver": "3.1.5", + "@swimlane/ngx-charts": "22.0.0", "@types/lodash-es": "4.17.4", "@types/plotly.js-basic-dist-min": "2.12.4", "ajv": "8.10.0", diff --git a/core/gui/proxy.config.json b/core/gui/proxy.config.json index 2702a9892c4..3ce084347a7 100755 --- a/core/gui/proxy.config.json +++ b/core/gui/proxy.config.json @@ -14,6 +14,16 @@ "secure": false, "changeOrigin": true }, + "/api/workflow-suggestion": { + "target": "http://0.0.0.0:9094", + "secure": false, + "changeOrigin": true + }, + "/api/data-cleaning-suggestion": { + "target": "http://0.0.0.0:9094", + "secure": false, + "changeOrigin": true + }, "/api/config/**": { "target": "http://localhost:9094", "secure": false, diff --git a/core/gui/src/app/app.module.ts b/core/gui/src/app/app.module.ts index 5087fcb0fc7..ed31c781f49 100644 --- a/core/gui/src/app/app.module.ts +++ b/core/gui/src/app/app.module.ts @@ -170,6 +170,10 @@ import { ComputingUnitSelectionComponent } from "./workspace/component/power-but import { NzSliderModule } from "ng-zorro-antd/slider"; import { AdminSettingsComponent } from "./dashboard/component/admin/settings/admin-settings.component"; import { catchError, of } from "rxjs"; +import { SuggestionFrameComponent } from "./workspace/component/result-panel/suggestion-frame/suggestion-frame.component"; +import { NzDescriptionsModule } from "ng-zorro-antd/descriptions"; +import { NgxChartsModule } from "@swimlane/ngx-charts"; +import { ColumnProfileFrameComponent } from "./workspace/component/left-panel/column-profile-frame/column-profile-frame.component"; registerLocaleData(en); @@ -261,6 +265,8 @@ registerLocaleData(en); HubSearchResultComponent, ComputingUnitSelectionComponent, AdminSettingsComponent, + SuggestionFrameComponent, + ColumnProfileFrameComponent, ], imports: [ BrowserModule, @@ -326,6 +332,8 @@ registerLocaleData(en); NzEmptyModule, NzDividerModule, NzProgressModule, + NzDescriptionsModule, + NgxChartsModule, ], providers: [ provideNzI18n(en_US), diff --git a/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/virtualidentity.ts b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/virtualidentity.ts new file mode 100644 index 00000000000..2f0f99195ff --- /dev/null +++ b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/virtualidentity.ts @@ -0,0 +1,531 @@ +// Code generated by protoc-gen-ts_proto. DO NOT EDIT. +// versions: +// protoc-gen-ts_proto v2.2.0 +// protoc v5.29.3 +// source: edu/uci/ics/amber/core/virtualidentity.proto + +/* eslint-disable */ +import { BinaryReader, BinaryWriter } from "@bufbuild/protobuf/wire"; + +export const protobufPackage = "edu.uci.ics.amber.core"; + +export interface WorkflowIdentity { + id: number; +} + +export interface ExecutionIdentity { + id: number; +} + +export interface ActorVirtualIdentity { + name: string; +} + +export interface ChannelIdentity { + fromWorkerId: ActorVirtualIdentity | undefined; + toWorkerId: ActorVirtualIdentity | undefined; + isControl: boolean; +} + +export interface OperatorIdentity { + id: string; +} + +export interface PhysicalOpIdentity { + logicalOpId: OperatorIdentity | undefined; + layerName: string; +} + +export interface ChannelMarkerIdentity { + id: string; +} + +function createBaseWorkflowIdentity(): WorkflowIdentity { + return { id: 0 }; +} + +export const WorkflowIdentity: MessageFns = { + encode(message: WorkflowIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== 0) { + writer.uint32(8).int64(message.id); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): WorkflowIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseWorkflowIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.id = longToNumber(reader.int64()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): WorkflowIdentity { + return { id: isSet(object.id) ? globalThis.Number(object.id) : 0 }; + }, + + toJSON(message: WorkflowIdentity): unknown { + const obj: any = {}; + if (message.id !== 0) { + obj.id = Math.round(message.id); + } + return obj; + }, + + create, I>>(base?: I): WorkflowIdentity { + return WorkflowIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): WorkflowIdentity { + const message = createBaseWorkflowIdentity(); + message.id = object.id ?? 0; + return message; + }, +}; + +function createBaseExecutionIdentity(): ExecutionIdentity { + return { id: 0 }; +} + +export const ExecutionIdentity: MessageFns = { + encode(message: ExecutionIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== 0) { + writer.uint32(8).int64(message.id); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ExecutionIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseExecutionIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.id = longToNumber(reader.int64()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ExecutionIdentity { + return { id: isSet(object.id) ? globalThis.Number(object.id) : 0 }; + }, + + toJSON(message: ExecutionIdentity): unknown { + const obj: any = {}; + if (message.id !== 0) { + obj.id = Math.round(message.id); + } + return obj; + }, + + create, I>>(base?: I): ExecutionIdentity { + return ExecutionIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ExecutionIdentity { + const message = createBaseExecutionIdentity(); + message.id = object.id ?? 0; + return message; + }, +}; + +function createBaseActorVirtualIdentity(): ActorVirtualIdentity { + return { name: "" }; +} + +export const ActorVirtualIdentity: MessageFns = { + encode(message: ActorVirtualIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.name !== "") { + writer.uint32(10).string(message.name); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ActorVirtualIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseActorVirtualIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.name = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ActorVirtualIdentity { + return { name: isSet(object.name) ? globalThis.String(object.name) : "" }; + }, + + toJSON(message: ActorVirtualIdentity): unknown { + const obj: any = {}; + if (message.name !== "") { + obj.name = message.name; + } + return obj; + }, + + create, I>>(base?: I): ActorVirtualIdentity { + return ActorVirtualIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ActorVirtualIdentity { + const message = createBaseActorVirtualIdentity(); + message.name = object.name ?? ""; + return message; + }, +}; + +function createBaseChannelIdentity(): ChannelIdentity { + return { fromWorkerId: undefined, toWorkerId: undefined, isControl: false }; +} + +export const ChannelIdentity: MessageFns = { + encode(message: ChannelIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.fromWorkerId !== undefined) { + ActorVirtualIdentity.encode(message.fromWorkerId, writer.uint32(10).fork()).join(); + } + if (message.toWorkerId !== undefined) { + ActorVirtualIdentity.encode(message.toWorkerId, writer.uint32(18).fork()).join(); + } + if (message.isControl !== false) { + writer.uint32(24).bool(message.isControl); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ChannelIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseChannelIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.fromWorkerId = ActorVirtualIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.toWorkerId = ActorVirtualIdentity.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.isControl = reader.bool(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ChannelIdentity { + return { + fromWorkerId: isSet(object.fromWorkerId) ? ActorVirtualIdentity.fromJSON(object.fromWorkerId) : undefined, + toWorkerId: isSet(object.toWorkerId) ? ActorVirtualIdentity.fromJSON(object.toWorkerId) : undefined, + isControl: isSet(object.isControl) ? globalThis.Boolean(object.isControl) : false, + }; + }, + + toJSON(message: ChannelIdentity): unknown { + const obj: any = {}; + if (message.fromWorkerId !== undefined) { + obj.fromWorkerId = ActorVirtualIdentity.toJSON(message.fromWorkerId); + } + if (message.toWorkerId !== undefined) { + obj.toWorkerId = ActorVirtualIdentity.toJSON(message.toWorkerId); + } + if (message.isControl !== false) { + obj.isControl = message.isControl; + } + return obj; + }, + + create, I>>(base?: I): ChannelIdentity { + return ChannelIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ChannelIdentity { + const message = createBaseChannelIdentity(); + message.fromWorkerId = (object.fromWorkerId !== undefined && object.fromWorkerId !== null) + ? ActorVirtualIdentity.fromPartial(object.fromWorkerId) + : undefined; + message.toWorkerId = (object.toWorkerId !== undefined && object.toWorkerId !== null) + ? ActorVirtualIdentity.fromPartial(object.toWorkerId) + : undefined; + message.isControl = object.isControl ?? false; + return message; + }, +}; + +function createBaseOperatorIdentity(): OperatorIdentity { + return { id: "" }; +} + +export const OperatorIdentity: MessageFns = { + encode(message: OperatorIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== "") { + writer.uint32(10).string(message.id); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): OperatorIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseOperatorIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.id = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): OperatorIdentity { + return { id: isSet(object.id) ? globalThis.String(object.id) : "" }; + }, + + toJSON(message: OperatorIdentity): unknown { + const obj: any = {}; + if (message.id !== "") { + obj.id = message.id; + } + return obj; + }, + + create, I>>(base?: I): OperatorIdentity { + return OperatorIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): OperatorIdentity { + const message = createBaseOperatorIdentity(); + message.id = object.id ?? ""; + return message; + }, +}; + +function createBasePhysicalOpIdentity(): PhysicalOpIdentity { + return { logicalOpId: undefined, layerName: "" }; +} + +export const PhysicalOpIdentity: MessageFns = { + encode(message: PhysicalOpIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.logicalOpId !== undefined) { + OperatorIdentity.encode(message.logicalOpId, writer.uint32(10).fork()).join(); + } + if (message.layerName !== "") { + writer.uint32(18).string(message.layerName); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): PhysicalOpIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePhysicalOpIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.logicalOpId = OperatorIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.layerName = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): PhysicalOpIdentity { + return { + logicalOpId: isSet(object.logicalOpId) ? OperatorIdentity.fromJSON(object.logicalOpId) : undefined, + layerName: isSet(object.layerName) ? globalThis.String(object.layerName) : "", + }; + }, + + toJSON(message: PhysicalOpIdentity): unknown { + const obj: any = {}; + if (message.logicalOpId !== undefined) { + obj.logicalOpId = OperatorIdentity.toJSON(message.logicalOpId); + } + if (message.layerName !== "") { + obj.layerName = message.layerName; + } + return obj; + }, + + create, I>>(base?: I): PhysicalOpIdentity { + return PhysicalOpIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PhysicalOpIdentity { + const message = createBasePhysicalOpIdentity(); + message.logicalOpId = (object.logicalOpId !== undefined && object.logicalOpId !== null) + ? OperatorIdentity.fromPartial(object.logicalOpId) + : undefined; + message.layerName = object.layerName ?? ""; + return message; + }, +}; + +function createBaseChannelMarkerIdentity(): ChannelMarkerIdentity { + return { id: "" }; +} + +export const ChannelMarkerIdentity: MessageFns = { + encode(message: ChannelMarkerIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== "") { + writer.uint32(10).string(message.id); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ChannelMarkerIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseChannelMarkerIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.id = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ChannelMarkerIdentity { + return { id: isSet(object.id) ? globalThis.String(object.id) : "" }; + }, + + toJSON(message: ChannelMarkerIdentity): unknown { + const obj: any = {}; + if (message.id !== "") { + obj.id = message.id; + } + return obj; + }, + + create, I>>(base?: I): ChannelMarkerIdentity { + return ChannelMarkerIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ChannelMarkerIdentity { + const message = createBaseChannelMarkerIdentity(); + message.id = object.id ?? ""; + return message; + }, +}; + +type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; + +export type DeepPartial = T extends Builtin ? T + : T extends globalThis.Array ? globalThis.Array> + : T extends ReadonlyArray ? ReadonlyArray> + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; + +type KeysOfUnion = T extends T ? keyof T : never; +export type Exact = P extends Builtin ? P + : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; + +function longToNumber(int64: { toString(): string }): number { + const num = globalThis.Number(int64.toString()); + if (num > globalThis.Number.MAX_SAFE_INTEGER) { + throw new globalThis.Error("Value is larger than Number.MAX_SAFE_INTEGER"); + } + if (num < globalThis.Number.MIN_SAFE_INTEGER) { + throw new globalThis.Error("Value is smaller than Number.MIN_SAFE_INTEGER"); + } + return num; +} + +function isSet(value: any): boolean { + return value !== null && value !== undefined; +} + +export interface MessageFns { + encode(message: T, writer?: BinaryWriter): BinaryWriter; + decode(input: BinaryReader | Uint8Array, length?: number): T; + fromJSON(object: any): T; + toJSON(message: T): unknown; + create, I>>(base?: I): T; + fromPartial, I>>(object: I): T; +} diff --git a/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/workflow.ts b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/workflow.ts new file mode 100644 index 00000000000..0a5ad0f3dec --- /dev/null +++ b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/core/workflow.ts @@ -0,0 +1,603 @@ +// Code generated by protoc-gen-ts_proto. DO NOT EDIT. +// versions: +// protoc-gen-ts_proto v2.2.0 +// protoc v5.29.3 +// source: edu/uci/ics/amber/core/workflow.proto + +/* eslint-disable */ +import { BinaryReader, BinaryWriter } from "@bufbuild/protobuf/wire"; +import { PhysicalOpIdentity } from "./virtualidentity"; + +export const protobufPackage = "edu.uci.ics.amber.core"; + +export interface PortIdentity { + id: number; + internal: boolean; +} + +export interface GlobalPortIdentity { + opId: PhysicalOpIdentity | undefined; + portId: PortIdentity | undefined; + input: boolean; +} + +export interface InputPort { + id: PortIdentity | undefined; + displayName: string; + allowMultiLinks: boolean; + dependencies: PortIdentity[]; +} + +export interface OutputPort { + id: PortIdentity | undefined; + displayName: string; + blocking: boolean; + mode: OutputPort_OutputMode; +} + +export enum OutputPort_OutputMode { + /** SET_SNAPSHOT - outputs complete result set snapshot for each update */ + SET_SNAPSHOT = 0, + /** SET_DELTA - outputs incremental result set delta for each update */ + SET_DELTA = 1, + /** + * SINGLE_SNAPSHOT - outputs a single snapshot for the entire execution, + * used explicitly to support visualization operators that may exceed the memory limit + * TODO: remove this mode after we have a better solution for output size limit + */ + SINGLE_SNAPSHOT = 2, + UNRECOGNIZED = -1, +} + +export function outputPort_OutputModeFromJSON(object: any): OutputPort_OutputMode { + switch (object) { + case 0: + case "SET_SNAPSHOT": + return OutputPort_OutputMode.SET_SNAPSHOT; + case 1: + case "SET_DELTA": + return OutputPort_OutputMode.SET_DELTA; + case 2: + case "SINGLE_SNAPSHOT": + return OutputPort_OutputMode.SINGLE_SNAPSHOT; + case -1: + case "UNRECOGNIZED": + default: + return OutputPort_OutputMode.UNRECOGNIZED; + } +} + +export function outputPort_OutputModeToJSON(object: OutputPort_OutputMode): string { + switch (object) { + case OutputPort_OutputMode.SET_SNAPSHOT: + return "SET_SNAPSHOT"; + case OutputPort_OutputMode.SET_DELTA: + return "SET_DELTA"; + case OutputPort_OutputMode.SINGLE_SNAPSHOT: + return "SINGLE_SNAPSHOT"; + case OutputPort_OutputMode.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + +export interface PhysicalLink { + fromOpId: PhysicalOpIdentity | undefined; + fromPortId: PortIdentity | undefined; + toOpId: PhysicalOpIdentity | undefined; + toPortId: PortIdentity | undefined; +} + +function createBasePortIdentity(): PortIdentity { + return { id: 0, internal: false }; +} + +export const PortIdentity: MessageFns = { + encode(message: PortIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== 0) { + writer.uint32(8).int32(message.id); + } + if (message.internal !== false) { + writer.uint32(16).bool(message.internal); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): PortIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePortIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.id = reader.int32(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.internal = reader.bool(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): PortIdentity { + return { + id: isSet(object.id) ? globalThis.Number(object.id) : 0, + internal: isSet(object.internal) ? globalThis.Boolean(object.internal) : false, + }; + }, + + toJSON(message: PortIdentity): unknown { + const obj: any = {}; + if (message.id !== 0) { + obj.id = Math.round(message.id); + } + if (message.internal !== false) { + obj.internal = message.internal; + } + return obj; + }, + + create, I>>(base?: I): PortIdentity { + return PortIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PortIdentity { + const message = createBasePortIdentity(); + message.id = object.id ?? 0; + message.internal = object.internal ?? false; + return message; + }, +}; + +function createBaseGlobalPortIdentity(): GlobalPortIdentity { + return { opId: undefined, portId: undefined, input: false }; +} + +export const GlobalPortIdentity: MessageFns = { + encode(message: GlobalPortIdentity, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.opId !== undefined) { + PhysicalOpIdentity.encode(message.opId, writer.uint32(10).fork()).join(); + } + if (message.portId !== undefined) { + PortIdentity.encode(message.portId, writer.uint32(18).fork()).join(); + } + if (message.input !== false) { + writer.uint32(24).bool(message.input); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): GlobalPortIdentity { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseGlobalPortIdentity(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.opId = PhysicalOpIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.portId = PortIdentity.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.input = reader.bool(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): GlobalPortIdentity { + return { + opId: isSet(object.opId) ? PhysicalOpIdentity.fromJSON(object.opId) : undefined, + portId: isSet(object.portId) ? PortIdentity.fromJSON(object.portId) : undefined, + input: isSet(object.input) ? globalThis.Boolean(object.input) : false, + }; + }, + + toJSON(message: GlobalPortIdentity): unknown { + const obj: any = {}; + if (message.opId !== undefined) { + obj.opId = PhysicalOpIdentity.toJSON(message.opId); + } + if (message.portId !== undefined) { + obj.portId = PortIdentity.toJSON(message.portId); + } + if (message.input !== false) { + obj.input = message.input; + } + return obj; + }, + + create, I>>(base?: I): GlobalPortIdentity { + return GlobalPortIdentity.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): GlobalPortIdentity { + const message = createBaseGlobalPortIdentity(); + message.opId = (object.opId !== undefined && object.opId !== null) + ? PhysicalOpIdentity.fromPartial(object.opId) + : undefined; + message.portId = (object.portId !== undefined && object.portId !== null) + ? PortIdentity.fromPartial(object.portId) + : undefined; + message.input = object.input ?? false; + return message; + }, +}; + +function createBaseInputPort(): InputPort { + return { id: undefined, displayName: "", allowMultiLinks: false, dependencies: [] }; +} + +export const InputPort: MessageFns = { + encode(message: InputPort, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== undefined) { + PortIdentity.encode(message.id, writer.uint32(10).fork()).join(); + } + if (message.displayName !== "") { + writer.uint32(18).string(message.displayName); + } + if (message.allowMultiLinks !== false) { + writer.uint32(24).bool(message.allowMultiLinks); + } + for (const v of message.dependencies) { + PortIdentity.encode(v!, writer.uint32(34).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): InputPort { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseInputPort(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.id = PortIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.displayName = reader.string(); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.allowMultiLinks = reader.bool(); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.dependencies.push(PortIdentity.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): InputPort { + return { + id: isSet(object.id) ? PortIdentity.fromJSON(object.id) : undefined, + displayName: isSet(object.displayName) ? globalThis.String(object.displayName) : "", + allowMultiLinks: isSet(object.allowMultiLinks) ? globalThis.Boolean(object.allowMultiLinks) : false, + dependencies: globalThis.Array.isArray(object?.dependencies) + ? object.dependencies.map((e: any) => PortIdentity.fromJSON(e)) + : [], + }; + }, + + toJSON(message: InputPort): unknown { + const obj: any = {}; + if (message.id !== undefined) { + obj.id = PortIdentity.toJSON(message.id); + } + if (message.displayName !== "") { + obj.displayName = message.displayName; + } + if (message.allowMultiLinks !== false) { + obj.allowMultiLinks = message.allowMultiLinks; + } + if (message.dependencies?.length) { + obj.dependencies = message.dependencies.map((e) => PortIdentity.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): InputPort { + return InputPort.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): InputPort { + const message = createBaseInputPort(); + message.id = (object.id !== undefined && object.id !== null) ? PortIdentity.fromPartial(object.id) : undefined; + message.displayName = object.displayName ?? ""; + message.allowMultiLinks = object.allowMultiLinks ?? false; + message.dependencies = object.dependencies?.map((e) => PortIdentity.fromPartial(e)) || []; + return message; + }, +}; + +function createBaseOutputPort(): OutputPort { + return { id: undefined, displayName: "", blocking: false, mode: 0 }; +} + +export const OutputPort: MessageFns = { + encode(message: OutputPort, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.id !== undefined) { + PortIdentity.encode(message.id, writer.uint32(10).fork()).join(); + } + if (message.displayName !== "") { + writer.uint32(18).string(message.displayName); + } + if (message.blocking !== false) { + writer.uint32(24).bool(message.blocking); + } + if (message.mode !== 0) { + writer.uint32(32).int32(message.mode); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): OutputPort { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseOutputPort(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.id = PortIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.displayName = reader.string(); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.blocking = reader.bool(); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.mode = reader.int32() as any; + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): OutputPort { + return { + id: isSet(object.id) ? PortIdentity.fromJSON(object.id) : undefined, + displayName: isSet(object.displayName) ? globalThis.String(object.displayName) : "", + blocking: isSet(object.blocking) ? globalThis.Boolean(object.blocking) : false, + mode: isSet(object.mode) ? outputPort_OutputModeFromJSON(object.mode) : 0, + }; + }, + + toJSON(message: OutputPort): unknown { + const obj: any = {}; + if (message.id !== undefined) { + obj.id = PortIdentity.toJSON(message.id); + } + if (message.displayName !== "") { + obj.displayName = message.displayName; + } + if (message.blocking !== false) { + obj.blocking = message.blocking; + } + if (message.mode !== 0) { + obj.mode = outputPort_OutputModeToJSON(message.mode); + } + return obj; + }, + + create, I>>(base?: I): OutputPort { + return OutputPort.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): OutputPort { + const message = createBaseOutputPort(); + message.id = (object.id !== undefined && object.id !== null) ? PortIdentity.fromPartial(object.id) : undefined; + message.displayName = object.displayName ?? ""; + message.blocking = object.blocking ?? false; + message.mode = object.mode ?? 0; + return message; + }, +}; + +function createBasePhysicalLink(): PhysicalLink { + return { fromOpId: undefined, fromPortId: undefined, toOpId: undefined, toPortId: undefined }; +} + +export const PhysicalLink: MessageFns = { + encode(message: PhysicalLink, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.fromOpId !== undefined) { + PhysicalOpIdentity.encode(message.fromOpId, writer.uint32(10).fork()).join(); + } + if (message.fromPortId !== undefined) { + PortIdentity.encode(message.fromPortId, writer.uint32(18).fork()).join(); + } + if (message.toOpId !== undefined) { + PhysicalOpIdentity.encode(message.toOpId, writer.uint32(26).fork()).join(); + } + if (message.toPortId !== undefined) { + PortIdentity.encode(message.toPortId, writer.uint32(34).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): PhysicalLink { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePhysicalLink(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.fromOpId = PhysicalOpIdentity.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.fromPortId = PortIdentity.decode(reader, reader.uint32()); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.toOpId = PhysicalOpIdentity.decode(reader, reader.uint32()); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.toPortId = PortIdentity.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): PhysicalLink { + return { + fromOpId: isSet(object.fromOpId) ? PhysicalOpIdentity.fromJSON(object.fromOpId) : undefined, + fromPortId: isSet(object.fromPortId) ? PortIdentity.fromJSON(object.fromPortId) : undefined, + toOpId: isSet(object.toOpId) ? PhysicalOpIdentity.fromJSON(object.toOpId) : undefined, + toPortId: isSet(object.toPortId) ? PortIdentity.fromJSON(object.toPortId) : undefined, + }; + }, + + toJSON(message: PhysicalLink): unknown { + const obj: any = {}; + if (message.fromOpId !== undefined) { + obj.fromOpId = PhysicalOpIdentity.toJSON(message.fromOpId); + } + if (message.fromPortId !== undefined) { + obj.fromPortId = PortIdentity.toJSON(message.fromPortId); + } + if (message.toOpId !== undefined) { + obj.toOpId = PhysicalOpIdentity.toJSON(message.toOpId); + } + if (message.toPortId !== undefined) { + obj.toPortId = PortIdentity.toJSON(message.toPortId); + } + return obj; + }, + + create, I>>(base?: I): PhysicalLink { + return PhysicalLink.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PhysicalLink { + const message = createBasePhysicalLink(); + message.fromOpId = (object.fromOpId !== undefined && object.fromOpId !== null) + ? PhysicalOpIdentity.fromPartial(object.fromOpId) + : undefined; + message.fromPortId = (object.fromPortId !== undefined && object.fromPortId !== null) + ? PortIdentity.fromPartial(object.fromPortId) + : undefined; + message.toOpId = (object.toOpId !== undefined && object.toOpId !== null) + ? PhysicalOpIdentity.fromPartial(object.toOpId) + : undefined; + message.toPortId = (object.toPortId !== undefined && object.toPortId !== null) + ? PortIdentity.fromPartial(object.toPortId) + : undefined; + return message; + }, +}; + +type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; + +export type DeepPartial = T extends Builtin ? T + : T extends globalThis.Array ? globalThis.Array> + : T extends ReadonlyArray ? ReadonlyArray> + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; + +type KeysOfUnion = T extends T ? keyof T : never; +export type Exact = P extends Builtin ? P + : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; + +function isSet(value: any): boolean { + return value !== null && value !== undefined; +} + +export interface MessageFns { + encode(message: T, writer?: BinaryWriter): BinaryWriter; + decode(input: BinaryReader | Uint8Array, length?: number): T; + fromJSON(object: any): T; + toJSON(message: T): unknown; + create, I>>(base?: I): T; + fromPartial, I>>(object: I): T; +} diff --git a/core/gui/src/app/common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile.ts b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile.ts new file mode 100644 index 00000000000..fa24a0bd998 --- /dev/null +++ b/core/gui/src/app/common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile.ts @@ -0,0 +1,1549 @@ +// Code generated by protoc-gen-ts_proto. DO NOT EDIT. +// versions: +// protoc-gen-ts_proto v2.2.0 +// protoc v5.29.3 +// source: edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto + +/* eslint-disable */ +import { BinaryReader, BinaryWriter } from "@bufbuild/protobuf/wire"; + +export const protobufPackage = "edu.uci.ics.amber.engine.architecture.worker"; + +export interface NumericMatrix { + /** length = rows * cols (row-major) */ + values: number[]; + rows: number; + cols: number; +} + +export interface GlobalProfile { + /** ---- basic counts ---- */ + samplesUsed: number; + columnCount: number; + rowCount: number; + /** ---- row null / uniqueness ---- */ + rowHasNullRatio: number; + rowIsNullRatio: number; + uniqueRowRatio: number; + duplicateRowCount: number; + /** ---- metadata ---- */ + fileType: string; + /** e.g. “utf-8” */ + encoding: string; + /** ---- pairwise stats ---- */ + correlationMatrix: NumericMatrix | undefined; + chi2Matrix: + | NumericMatrix + | undefined; + /** ---- schema map: column-name -> indices (to mirror DataProfiler) --- */ + profileSchema: { [key: string]: ColumnIndexList }; + times: GlobalProfile_Times | undefined; +} + +export interface GlobalProfile_ProfileSchemaEntry { + key: string; + value: ColumnIndexList | undefined; +} + +/** ---- timing ---- */ +export interface GlobalProfile_Times { + rowStatsMs: number; +} + +/** helper for profile_schema */ +export interface ColumnIndexList { + indices: number[]; +} + +export interface ColumnStatistics { + /** ---- numeric summaries (nullable when not applicable) ---- */ + min: number; + max: number; + median: number; + mean: number; + variance: number; + stddev: number; + skewness: number; + kurtosis: number; + sum: number; + /** ---- distribution ---- */ + quantiles: number[]; + numZeros: number; + numNegatives: number; + /** ---- uniqueness / cardinality ---- */ + uniqueCount: number; + uniqueRatio: number; + /** ---- categorical helpers ---- */ + categorical: boolean; + categoricalCount: { [key: string]: number }; + /** ---- nulls ---- */ + nullCount: number; + nullTypes: string[]; + /** ---- data-type representation share (DataProfiler style) ---- */ + dataTypeRepresentation: { [key: string]: number }; +} + +export interface ColumnStatistics_CategoricalCountEntry { + key: string; + value: number; +} + +export interface ColumnStatistics_DataTypeRepresentationEntry { + key: string; + value: number; +} + +export interface ColumnProfile { + /** identity */ + columnName: string; + /** “string”, “int”, “float”, … */ + dataType: string; + dataLabel: string; + /** quick hints */ + categorical: boolean; + /** “random”, “ascending”, “constant value” */ + order: string; + /** examples */ + samples: string[]; + /** heavy stats */ + statistics: ColumnStatistics | undefined; +} + +export interface TableProfile { + globalProfile: GlobalProfile | undefined; + columnProfiles: ColumnProfile[]; +} + +function createBaseNumericMatrix(): NumericMatrix { + return { values: [], rows: 0, cols: 0 }; +} + +export const NumericMatrix: MessageFns = { + encode(message: NumericMatrix, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + writer.uint32(10).fork(); + for (const v of message.values) { + writer.double(v); + } + writer.join(); + if (message.rows !== 0) { + writer.uint32(16).uint32(message.rows); + } + if (message.cols !== 0) { + writer.uint32(24).uint32(message.cols); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): NumericMatrix { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseNumericMatrix(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag === 9) { + message.values.push(reader.double()); + + continue; + } + + if (tag === 10) { + const end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) { + message.values.push(reader.double()); + } + + continue; + } + + break; + case 2: + if (tag !== 16) { + break; + } + + message.rows = reader.uint32(); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.cols = reader.uint32(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): NumericMatrix { + return { + values: globalThis.Array.isArray(object?.values) ? object.values.map((e: any) => globalThis.Number(e)) : [], + rows: isSet(object.rows) ? globalThis.Number(object.rows) : 0, + cols: isSet(object.cols) ? globalThis.Number(object.cols) : 0, + }; + }, + + toJSON(message: NumericMatrix): unknown { + const obj: any = {}; + if (message.values?.length) { + obj.values = message.values; + } + if (message.rows !== 0) { + obj.rows = Math.round(message.rows); + } + if (message.cols !== 0) { + obj.cols = Math.round(message.cols); + } + return obj; + }, + + create, I>>(base?: I): NumericMatrix { + return NumericMatrix.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): NumericMatrix { + const message = createBaseNumericMatrix(); + message.values = object.values?.map((e) => e) || []; + message.rows = object.rows ?? 0; + message.cols = object.cols ?? 0; + return message; + }, +}; + +function createBaseGlobalProfile(): GlobalProfile { + return { + samplesUsed: 0, + columnCount: 0, + rowCount: 0, + rowHasNullRatio: 0, + rowIsNullRatio: 0, + uniqueRowRatio: 0, + duplicateRowCount: 0, + fileType: "", + encoding: "", + correlationMatrix: undefined, + chi2Matrix: undefined, + profileSchema: {}, + times: undefined, + }; +} + +export const GlobalProfile: MessageFns = { + encode(message: GlobalProfile, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.samplesUsed !== 0) { + writer.uint32(8).uint64(message.samplesUsed); + } + if (message.columnCount !== 0) { + writer.uint32(16).uint64(message.columnCount); + } + if (message.rowCount !== 0) { + writer.uint32(24).uint64(message.rowCount); + } + if (message.rowHasNullRatio !== 0) { + writer.uint32(33).double(message.rowHasNullRatio); + } + if (message.rowIsNullRatio !== 0) { + writer.uint32(41).double(message.rowIsNullRatio); + } + if (message.uniqueRowRatio !== 0) { + writer.uint32(49).double(message.uniqueRowRatio); + } + if (message.duplicateRowCount !== 0) { + writer.uint32(56).uint64(message.duplicateRowCount); + } + if (message.fileType !== "") { + writer.uint32(66).string(message.fileType); + } + if (message.encoding !== "") { + writer.uint32(74).string(message.encoding); + } + if (message.correlationMatrix !== undefined) { + NumericMatrix.encode(message.correlationMatrix, writer.uint32(82).fork()).join(); + } + if (message.chi2Matrix !== undefined) { + NumericMatrix.encode(message.chi2Matrix, writer.uint32(90).fork()).join(); + } + Object.entries(message.profileSchema).forEach(([key, value]) => { + GlobalProfile_ProfileSchemaEntry.encode({ key: key as any, value }, writer.uint32(98).fork()).join(); + }); + if (message.times !== undefined) { + GlobalProfile_Times.encode(message.times, writer.uint32(106).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): GlobalProfile { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseGlobalProfile(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.samplesUsed = longToNumber(reader.uint64()); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.columnCount = longToNumber(reader.uint64()); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.rowCount = longToNumber(reader.uint64()); + continue; + case 4: + if (tag !== 33) { + break; + } + + message.rowHasNullRatio = reader.double(); + continue; + case 5: + if (tag !== 41) { + break; + } + + message.rowIsNullRatio = reader.double(); + continue; + case 6: + if (tag !== 49) { + break; + } + + message.uniqueRowRatio = reader.double(); + continue; + case 7: + if (tag !== 56) { + break; + } + + message.duplicateRowCount = longToNumber(reader.uint64()); + continue; + case 8: + if (tag !== 66) { + break; + } + + message.fileType = reader.string(); + continue; + case 9: + if (tag !== 74) { + break; + } + + message.encoding = reader.string(); + continue; + case 10: + if (tag !== 82) { + break; + } + + message.correlationMatrix = NumericMatrix.decode(reader, reader.uint32()); + continue; + case 11: + if (tag !== 90) { + break; + } + + message.chi2Matrix = NumericMatrix.decode(reader, reader.uint32()); + continue; + case 12: + if (tag !== 98) { + break; + } + + const entry12 = GlobalProfile_ProfileSchemaEntry.decode(reader, reader.uint32()); + if (entry12.value !== undefined) { + message.profileSchema[entry12.key] = entry12.value; + } + continue; + case 13: + if (tag !== 106) { + break; + } + + message.times = GlobalProfile_Times.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): GlobalProfile { + return { + samplesUsed: isSet(object.samplesUsed) ? globalThis.Number(object.samplesUsed) : 0, + columnCount: isSet(object.columnCount) ? globalThis.Number(object.columnCount) : 0, + rowCount: isSet(object.rowCount) ? globalThis.Number(object.rowCount) : 0, + rowHasNullRatio: isSet(object.rowHasNullRatio) ? globalThis.Number(object.rowHasNullRatio) : 0, + rowIsNullRatio: isSet(object.rowIsNullRatio) ? globalThis.Number(object.rowIsNullRatio) : 0, + uniqueRowRatio: isSet(object.uniqueRowRatio) ? globalThis.Number(object.uniqueRowRatio) : 0, + duplicateRowCount: isSet(object.duplicateRowCount) ? globalThis.Number(object.duplicateRowCount) : 0, + fileType: isSet(object.fileType) ? globalThis.String(object.fileType) : "", + encoding: isSet(object.encoding) ? globalThis.String(object.encoding) : "", + correlationMatrix: isSet(object.correlationMatrix) ? NumericMatrix.fromJSON(object.correlationMatrix) : undefined, + chi2Matrix: isSet(object.chi2Matrix) ? NumericMatrix.fromJSON(object.chi2Matrix) : undefined, + profileSchema: isObject(object.profileSchema) + ? Object.entries(object.profileSchema).reduce<{ [key: string]: ColumnIndexList }>((acc, [key, value]) => { + acc[key] = ColumnIndexList.fromJSON(value); + return acc; + }, {}) + : {}, + times: isSet(object.times) ? GlobalProfile_Times.fromJSON(object.times) : undefined, + }; + }, + + toJSON(message: GlobalProfile): unknown { + const obj: any = {}; + if (message.samplesUsed !== 0) { + obj.samplesUsed = Math.round(message.samplesUsed); + } + if (message.columnCount !== 0) { + obj.columnCount = Math.round(message.columnCount); + } + if (message.rowCount !== 0) { + obj.rowCount = Math.round(message.rowCount); + } + if (message.rowHasNullRatio !== 0) { + obj.rowHasNullRatio = message.rowHasNullRatio; + } + if (message.rowIsNullRatio !== 0) { + obj.rowIsNullRatio = message.rowIsNullRatio; + } + if (message.uniqueRowRatio !== 0) { + obj.uniqueRowRatio = message.uniqueRowRatio; + } + if (message.duplicateRowCount !== 0) { + obj.duplicateRowCount = Math.round(message.duplicateRowCount); + } + if (message.fileType !== "") { + obj.fileType = message.fileType; + } + if (message.encoding !== "") { + obj.encoding = message.encoding; + } + if (message.correlationMatrix !== undefined) { + obj.correlationMatrix = NumericMatrix.toJSON(message.correlationMatrix); + } + if (message.chi2Matrix !== undefined) { + obj.chi2Matrix = NumericMatrix.toJSON(message.chi2Matrix); + } + if (message.profileSchema) { + const entries = Object.entries(message.profileSchema); + if (entries.length > 0) { + obj.profileSchema = {}; + entries.forEach(([k, v]) => { + obj.profileSchema[k] = ColumnIndexList.toJSON(v); + }); + } + } + if (message.times !== undefined) { + obj.times = GlobalProfile_Times.toJSON(message.times); + } + return obj; + }, + + create, I>>(base?: I): GlobalProfile { + return GlobalProfile.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): GlobalProfile { + const message = createBaseGlobalProfile(); + message.samplesUsed = object.samplesUsed ?? 0; + message.columnCount = object.columnCount ?? 0; + message.rowCount = object.rowCount ?? 0; + message.rowHasNullRatio = object.rowHasNullRatio ?? 0; + message.rowIsNullRatio = object.rowIsNullRatio ?? 0; + message.uniqueRowRatio = object.uniqueRowRatio ?? 0; + message.duplicateRowCount = object.duplicateRowCount ?? 0; + message.fileType = object.fileType ?? ""; + message.encoding = object.encoding ?? ""; + message.correlationMatrix = (object.correlationMatrix !== undefined && object.correlationMatrix !== null) + ? NumericMatrix.fromPartial(object.correlationMatrix) + : undefined; + message.chi2Matrix = (object.chi2Matrix !== undefined && object.chi2Matrix !== null) + ? NumericMatrix.fromPartial(object.chi2Matrix) + : undefined; + message.profileSchema = Object.entries(object.profileSchema ?? {}).reduce<{ [key: string]: ColumnIndexList }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = ColumnIndexList.fromPartial(value); + } + return acc; + }, + {}, + ); + message.times = (object.times !== undefined && object.times !== null) + ? GlobalProfile_Times.fromPartial(object.times) + : undefined; + return message; + }, +}; + +function createBaseGlobalProfile_ProfileSchemaEntry(): GlobalProfile_ProfileSchemaEntry { + return { key: "", value: undefined }; +} + +export const GlobalProfile_ProfileSchemaEntry: MessageFns = { + encode(message: GlobalProfile_ProfileSchemaEntry, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== undefined) { + ColumnIndexList.encode(message.value, writer.uint32(18).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): GlobalProfile_ProfileSchemaEntry { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseGlobalProfile_ProfileSchemaEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = ColumnIndexList.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): GlobalProfile_ProfileSchemaEntry { + return { + key: isSet(object.key) ? globalThis.String(object.key) : "", + value: isSet(object.value) ? ColumnIndexList.fromJSON(object.value) : undefined, + }; + }, + + toJSON(message: GlobalProfile_ProfileSchemaEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== undefined) { + obj.value = ColumnIndexList.toJSON(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): GlobalProfile_ProfileSchemaEntry { + return GlobalProfile_ProfileSchemaEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): GlobalProfile_ProfileSchemaEntry { + const message = createBaseGlobalProfile_ProfileSchemaEntry(); + message.key = object.key ?? ""; + message.value = (object.value !== undefined && object.value !== null) + ? ColumnIndexList.fromPartial(object.value) + : undefined; + return message; + }, +}; + +function createBaseGlobalProfile_Times(): GlobalProfile_Times { + return { rowStatsMs: 0 }; +} + +export const GlobalProfile_Times: MessageFns = { + encode(message: GlobalProfile_Times, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.rowStatsMs !== 0) { + writer.uint32(9).double(message.rowStatsMs); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): GlobalProfile_Times { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseGlobalProfile_Times(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 9) { + break; + } + + message.rowStatsMs = reader.double(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): GlobalProfile_Times { + return { rowStatsMs: isSet(object.rowStatsMs) ? globalThis.Number(object.rowStatsMs) : 0 }; + }, + + toJSON(message: GlobalProfile_Times): unknown { + const obj: any = {}; + if (message.rowStatsMs !== 0) { + obj.rowStatsMs = message.rowStatsMs; + } + return obj; + }, + + create, I>>(base?: I): GlobalProfile_Times { + return GlobalProfile_Times.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): GlobalProfile_Times { + const message = createBaseGlobalProfile_Times(); + message.rowStatsMs = object.rowStatsMs ?? 0; + return message; + }, +}; + +function createBaseColumnIndexList(): ColumnIndexList { + return { indices: [] }; +} + +export const ColumnIndexList: MessageFns = { + encode(message: ColumnIndexList, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + writer.uint32(10).fork(); + for (const v of message.indices) { + writer.uint32(v); + } + writer.join(); + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ColumnIndexList { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseColumnIndexList(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag === 8) { + message.indices.push(reader.uint32()); + + continue; + } + + if (tag === 10) { + const end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) { + message.indices.push(reader.uint32()); + } + + continue; + } + + break; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ColumnIndexList { + return { + indices: globalThis.Array.isArray(object?.indices) ? object.indices.map((e: any) => globalThis.Number(e)) : [], + }; + }, + + toJSON(message: ColumnIndexList): unknown { + const obj: any = {}; + if (message.indices?.length) { + obj.indices = message.indices.map((e) => Math.round(e)); + } + return obj; + }, + + create, I>>(base?: I): ColumnIndexList { + return ColumnIndexList.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ColumnIndexList { + const message = createBaseColumnIndexList(); + message.indices = object.indices?.map((e) => e) || []; + return message; + }, +}; + +function createBaseColumnStatistics(): ColumnStatistics { + return { + min: 0, + max: 0, + median: 0, + mean: 0, + variance: 0, + stddev: 0, + skewness: 0, + kurtosis: 0, + sum: 0, + quantiles: [], + numZeros: 0, + numNegatives: 0, + uniqueCount: 0, + uniqueRatio: 0, + categorical: false, + categoricalCount: {}, + nullCount: 0, + nullTypes: [], + dataTypeRepresentation: {}, + }; +} + +export const ColumnStatistics: MessageFns = { + encode(message: ColumnStatistics, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.min !== 0) { + writer.uint32(9).double(message.min); + } + if (message.max !== 0) { + writer.uint32(17).double(message.max); + } + if (message.median !== 0) { + writer.uint32(25).double(message.median); + } + if (message.mean !== 0) { + writer.uint32(33).double(message.mean); + } + if (message.variance !== 0) { + writer.uint32(41).double(message.variance); + } + if (message.stddev !== 0) { + writer.uint32(49).double(message.stddev); + } + if (message.skewness !== 0) { + writer.uint32(57).double(message.skewness); + } + if (message.kurtosis !== 0) { + writer.uint32(65).double(message.kurtosis); + } + if (message.sum !== 0) { + writer.uint32(73).double(message.sum); + } + writer.uint32(82).fork(); + for (const v of message.quantiles) { + writer.double(v); + } + writer.join(); + if (message.numZeros !== 0) { + writer.uint32(88).uint64(message.numZeros); + } + if (message.numNegatives !== 0) { + writer.uint32(96).uint64(message.numNegatives); + } + if (message.uniqueCount !== 0) { + writer.uint32(104).uint64(message.uniqueCount); + } + if (message.uniqueRatio !== 0) { + writer.uint32(113).double(message.uniqueRatio); + } + if (message.categorical !== false) { + writer.uint32(120).bool(message.categorical); + } + Object.entries(message.categoricalCount).forEach(([key, value]) => { + ColumnStatistics_CategoricalCountEntry.encode({ key: key as any, value }, writer.uint32(130).fork()).join(); + }); + if (message.nullCount !== 0) { + writer.uint32(136).uint64(message.nullCount); + } + for (const v of message.nullTypes) { + writer.uint32(146).string(v!); + } + Object.entries(message.dataTypeRepresentation).forEach(([key, value]) => { + ColumnStatistics_DataTypeRepresentationEntry.encode({ key: key as any, value }, writer.uint32(154).fork()).join(); + }); + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ColumnStatistics { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseColumnStatistics(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 9) { + break; + } + + message.min = reader.double(); + continue; + case 2: + if (tag !== 17) { + break; + } + + message.max = reader.double(); + continue; + case 3: + if (tag !== 25) { + break; + } + + message.median = reader.double(); + continue; + case 4: + if (tag !== 33) { + break; + } + + message.mean = reader.double(); + continue; + case 5: + if (tag !== 41) { + break; + } + + message.variance = reader.double(); + continue; + case 6: + if (tag !== 49) { + break; + } + + message.stddev = reader.double(); + continue; + case 7: + if (tag !== 57) { + break; + } + + message.skewness = reader.double(); + continue; + case 8: + if (tag !== 65) { + break; + } + + message.kurtosis = reader.double(); + continue; + case 9: + if (tag !== 73) { + break; + } + + message.sum = reader.double(); + continue; + case 10: + if (tag === 81) { + message.quantiles.push(reader.double()); + + continue; + } + + if (tag === 82) { + const end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) { + message.quantiles.push(reader.double()); + } + + continue; + } + + break; + case 11: + if (tag !== 88) { + break; + } + + message.numZeros = longToNumber(reader.uint64()); + continue; + case 12: + if (tag !== 96) { + break; + } + + message.numNegatives = longToNumber(reader.uint64()); + continue; + case 13: + if (tag !== 104) { + break; + } + + message.uniqueCount = longToNumber(reader.uint64()); + continue; + case 14: + if (tag !== 113) { + break; + } + + message.uniqueRatio = reader.double(); + continue; + case 15: + if (tag !== 120) { + break; + } + + message.categorical = reader.bool(); + continue; + case 16: + if (tag !== 130) { + break; + } + + const entry16 = ColumnStatistics_CategoricalCountEntry.decode(reader, reader.uint32()); + if (entry16.value !== undefined) { + message.categoricalCount[entry16.key] = entry16.value; + } + continue; + case 17: + if (tag !== 136) { + break; + } + + message.nullCount = longToNumber(reader.uint64()); + continue; + case 18: + if (tag !== 146) { + break; + } + + message.nullTypes.push(reader.string()); + continue; + case 19: + if (tag !== 154) { + break; + } + + const entry19 = ColumnStatistics_DataTypeRepresentationEntry.decode(reader, reader.uint32()); + if (entry19.value !== undefined) { + message.dataTypeRepresentation[entry19.key] = entry19.value; + } + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ColumnStatistics { + return { + min: isSet(object.min) ? globalThis.Number(object.min) : 0, + max: isSet(object.max) ? globalThis.Number(object.max) : 0, + median: isSet(object.median) ? globalThis.Number(object.median) : 0, + mean: isSet(object.mean) ? globalThis.Number(object.mean) : 0, + variance: isSet(object.variance) ? globalThis.Number(object.variance) : 0, + stddev: isSet(object.stddev) ? globalThis.Number(object.stddev) : 0, + skewness: isSet(object.skewness) ? globalThis.Number(object.skewness) : 0, + kurtosis: isSet(object.kurtosis) ? globalThis.Number(object.kurtosis) : 0, + sum: isSet(object.sum) ? globalThis.Number(object.sum) : 0, + quantiles: globalThis.Array.isArray(object?.quantiles) + ? object.quantiles.map((e: any) => globalThis.Number(e)) + : [], + numZeros: isSet(object.numZeros) ? globalThis.Number(object.numZeros) : 0, + numNegatives: isSet(object.numNegatives) ? globalThis.Number(object.numNegatives) : 0, + uniqueCount: isSet(object.uniqueCount) ? globalThis.Number(object.uniqueCount) : 0, + uniqueRatio: isSet(object.uniqueRatio) ? globalThis.Number(object.uniqueRatio) : 0, + categorical: isSet(object.categorical) ? globalThis.Boolean(object.categorical) : false, + categoricalCount: isObject(object.categoricalCount) + ? Object.entries(object.categoricalCount).reduce<{ [key: string]: number }>((acc, [key, value]) => { + acc[key] = Number(value); + return acc; + }, {}) + : {}, + nullCount: isSet(object.nullCount) ? globalThis.Number(object.nullCount) : 0, + nullTypes: globalThis.Array.isArray(object?.nullTypes) + ? object.nullTypes.map((e: any) => globalThis.String(e)) + : [], + dataTypeRepresentation: isObject(object.dataTypeRepresentation) + ? Object.entries(object.dataTypeRepresentation).reduce<{ [key: string]: number }>((acc, [key, value]) => { + acc[key] = Number(value); + return acc; + }, {}) + : {}, + }; + }, + + toJSON(message: ColumnStatistics): unknown { + const obj: any = {}; + if (message.min !== 0) { + obj.min = message.min; + } + if (message.max !== 0) { + obj.max = message.max; + } + if (message.median !== 0) { + obj.median = message.median; + } + if (message.mean !== 0) { + obj.mean = message.mean; + } + if (message.variance !== 0) { + obj.variance = message.variance; + } + if (message.stddev !== 0) { + obj.stddev = message.stddev; + } + if (message.skewness !== 0) { + obj.skewness = message.skewness; + } + if (message.kurtosis !== 0) { + obj.kurtosis = message.kurtosis; + } + if (message.sum !== 0) { + obj.sum = message.sum; + } + if (message.quantiles?.length) { + obj.quantiles = message.quantiles; + } + if (message.numZeros !== 0) { + obj.numZeros = Math.round(message.numZeros); + } + if (message.numNegatives !== 0) { + obj.numNegatives = Math.round(message.numNegatives); + } + if (message.uniqueCount !== 0) { + obj.uniqueCount = Math.round(message.uniqueCount); + } + if (message.uniqueRatio !== 0) { + obj.uniqueRatio = message.uniqueRatio; + } + if (message.categorical !== false) { + obj.categorical = message.categorical; + } + if (message.categoricalCount) { + const entries = Object.entries(message.categoricalCount); + if (entries.length > 0) { + obj.categoricalCount = {}; + entries.forEach(([k, v]) => { + obj.categoricalCount[k] = Math.round(v); + }); + } + } + if (message.nullCount !== 0) { + obj.nullCount = Math.round(message.nullCount); + } + if (message.nullTypes?.length) { + obj.nullTypes = message.nullTypes; + } + if (message.dataTypeRepresentation) { + const entries = Object.entries(message.dataTypeRepresentation); + if (entries.length > 0) { + obj.dataTypeRepresentation = {}; + entries.forEach(([k, v]) => { + obj.dataTypeRepresentation[k] = v; + }); + } + } + return obj; + }, + + create, I>>(base?: I): ColumnStatistics { + return ColumnStatistics.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ColumnStatistics { + const message = createBaseColumnStatistics(); + message.min = object.min ?? 0; + message.max = object.max ?? 0; + message.median = object.median ?? 0; + message.mean = object.mean ?? 0; + message.variance = object.variance ?? 0; + message.stddev = object.stddev ?? 0; + message.skewness = object.skewness ?? 0; + message.kurtosis = object.kurtosis ?? 0; + message.sum = object.sum ?? 0; + message.quantiles = object.quantiles?.map((e) => e) || []; + message.numZeros = object.numZeros ?? 0; + message.numNegatives = object.numNegatives ?? 0; + message.uniqueCount = object.uniqueCount ?? 0; + message.uniqueRatio = object.uniqueRatio ?? 0; + message.categorical = object.categorical ?? false; + message.categoricalCount = Object.entries(object.categoricalCount ?? {}).reduce<{ [key: string]: number }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = globalThis.Number(value); + } + return acc; + }, + {}, + ); + message.nullCount = object.nullCount ?? 0; + message.nullTypes = object.nullTypes?.map((e) => e) || []; + message.dataTypeRepresentation = Object.entries(object.dataTypeRepresentation ?? {}).reduce< + { [key: string]: number } + >((acc, [key, value]) => { + if (value !== undefined) { + acc[key] = globalThis.Number(value); + } + return acc; + }, {}); + return message; + }, +}; + +function createBaseColumnStatistics_CategoricalCountEntry(): ColumnStatistics_CategoricalCountEntry { + return { key: "", value: 0 }; +} + +export const ColumnStatistics_CategoricalCountEntry: MessageFns = { + encode(message: ColumnStatistics_CategoricalCountEntry, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== 0) { + writer.uint32(16).uint64(message.value); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ColumnStatistics_CategoricalCountEntry { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseColumnStatistics_CategoricalCountEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.value = longToNumber(reader.uint64()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ColumnStatistics_CategoricalCountEntry { + return { + key: isSet(object.key) ? globalThis.String(object.key) : "", + value: isSet(object.value) ? globalThis.Number(object.value) : 0, + }; + }, + + toJSON(message: ColumnStatistics_CategoricalCountEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== 0) { + obj.value = Math.round(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): ColumnStatistics_CategoricalCountEntry { + return ColumnStatistics_CategoricalCountEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): ColumnStatistics_CategoricalCountEntry { + const message = createBaseColumnStatistics_CategoricalCountEntry(); + message.key = object.key ?? ""; + message.value = object.value ?? 0; + return message; + }, +}; + +function createBaseColumnStatistics_DataTypeRepresentationEntry(): ColumnStatistics_DataTypeRepresentationEntry { + return { key: "", value: 0 }; +} + +export const ColumnStatistics_DataTypeRepresentationEntry: MessageFns = { + encode( + message: ColumnStatistics_DataTypeRepresentationEntry, + writer: BinaryWriter = new BinaryWriter(), + ): BinaryWriter { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== 0) { + writer.uint32(17).double(message.value); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ColumnStatistics_DataTypeRepresentationEntry { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseColumnStatistics_DataTypeRepresentationEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 17) { + break; + } + + message.value = reader.double(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ColumnStatistics_DataTypeRepresentationEntry { + return { + key: isSet(object.key) ? globalThis.String(object.key) : "", + value: isSet(object.value) ? globalThis.Number(object.value) : 0, + }; + }, + + toJSON(message: ColumnStatistics_DataTypeRepresentationEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== 0) { + obj.value = message.value; + } + return obj; + }, + + create, I>>( + base?: I, + ): ColumnStatistics_DataTypeRepresentationEntry { + return ColumnStatistics_DataTypeRepresentationEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): ColumnStatistics_DataTypeRepresentationEntry { + const message = createBaseColumnStatistics_DataTypeRepresentationEntry(); + message.key = object.key ?? ""; + message.value = object.value ?? 0; + return message; + }, +}; + +function createBaseColumnProfile(): ColumnProfile { + return { + columnName: "", + dataType: "", + dataLabel: "", + categorical: false, + order: "", + samples: [], + statistics: undefined, + }; +} + +export const ColumnProfile: MessageFns = { + encode(message: ColumnProfile, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.columnName !== "") { + writer.uint32(10).string(message.columnName); + } + if (message.dataType !== "") { + writer.uint32(18).string(message.dataType); + } + if (message.dataLabel !== "") { + writer.uint32(26).string(message.dataLabel); + } + if (message.categorical !== false) { + writer.uint32(32).bool(message.categorical); + } + if (message.order !== "") { + writer.uint32(42).string(message.order); + } + for (const v of message.samples) { + writer.uint32(50).string(v!); + } + if (message.statistics !== undefined) { + ColumnStatistics.encode(message.statistics, writer.uint32(58).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): ColumnProfile { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseColumnProfile(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.columnName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.dataType = reader.string(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.dataLabel = reader.string(); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.categorical = reader.bool(); + continue; + case 5: + if (tag !== 42) { + break; + } + + message.order = reader.string(); + continue; + case 6: + if (tag !== 50) { + break; + } + + message.samples.push(reader.string()); + continue; + case 7: + if (tag !== 58) { + break; + } + + message.statistics = ColumnStatistics.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): ColumnProfile { + return { + columnName: isSet(object.columnName) ? globalThis.String(object.columnName) : "", + dataType: isSet(object.dataType) ? globalThis.String(object.dataType) : "", + dataLabel: isSet(object.dataLabel) ? globalThis.String(object.dataLabel) : "", + categorical: isSet(object.categorical) ? globalThis.Boolean(object.categorical) : false, + order: isSet(object.order) ? globalThis.String(object.order) : "", + samples: globalThis.Array.isArray(object?.samples) ? object.samples.map((e: any) => globalThis.String(e)) : [], + statistics: isSet(object.statistics) ? ColumnStatistics.fromJSON(object.statistics) : undefined, + }; + }, + + toJSON(message: ColumnProfile): unknown { + const obj: any = {}; + if (message.columnName !== "") { + obj.columnName = message.columnName; + } + if (message.dataType !== "") { + obj.dataType = message.dataType; + } + if (message.dataLabel !== "") { + obj.dataLabel = message.dataLabel; + } + if (message.categorical !== false) { + obj.categorical = message.categorical; + } + if (message.order !== "") { + obj.order = message.order; + } + if (message.samples?.length) { + obj.samples = message.samples; + } + if (message.statistics !== undefined) { + obj.statistics = ColumnStatistics.toJSON(message.statistics); + } + return obj; + }, + + create, I>>(base?: I): ColumnProfile { + return ColumnProfile.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ColumnProfile { + const message = createBaseColumnProfile(); + message.columnName = object.columnName ?? ""; + message.dataType = object.dataType ?? ""; + message.dataLabel = object.dataLabel ?? ""; + message.categorical = object.categorical ?? false; + message.order = object.order ?? ""; + message.samples = object.samples?.map((e) => e) || []; + message.statistics = (object.statistics !== undefined && object.statistics !== null) + ? ColumnStatistics.fromPartial(object.statistics) + : undefined; + return message; + }, +}; + +function createBaseTableProfile(): TableProfile { + return { globalProfile: undefined, columnProfiles: [] }; +} + +export const TableProfile: MessageFns = { + encode(message: TableProfile, writer: BinaryWriter = new BinaryWriter()): BinaryWriter { + if (message.globalProfile !== undefined) { + GlobalProfile.encode(message.globalProfile, writer.uint32(10).fork()).join(); + } + for (const v of message.columnProfiles) { + ColumnProfile.encode(v!, writer.uint32(18).fork()).join(); + } + return writer; + }, + + decode(input: BinaryReader | Uint8Array, length?: number): TableProfile { + const reader = input instanceof BinaryReader ? input : new BinaryReader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseTableProfile(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.globalProfile = GlobalProfile.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.columnProfiles.push(ColumnProfile.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skip(tag & 7); + } + return message; + }, + + fromJSON(object: any): TableProfile { + return { + globalProfile: isSet(object.globalProfile) ? GlobalProfile.fromJSON(object.globalProfile) : undefined, + columnProfiles: globalThis.Array.isArray(object?.columnProfiles) + ? object.columnProfiles.map((e: any) => ColumnProfile.fromJSON(e)) + : [], + }; + }, + + toJSON(message: TableProfile): unknown { + const obj: any = {}; + if (message.globalProfile !== undefined) { + obj.globalProfile = GlobalProfile.toJSON(message.globalProfile); + } + if (message.columnProfiles?.length) { + obj.columnProfiles = message.columnProfiles.map((e) => ColumnProfile.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): TableProfile { + return TableProfile.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): TableProfile { + const message = createBaseTableProfile(); + message.globalProfile = (object.globalProfile !== undefined && object.globalProfile !== null) + ? GlobalProfile.fromPartial(object.globalProfile) + : undefined; + message.columnProfiles = object.columnProfiles?.map((e) => ColumnProfile.fromPartial(e)) || []; + return message; + }, +}; + +type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; + +export type DeepPartial = T extends Builtin ? T + : T extends globalThis.Array ? globalThis.Array> + : T extends ReadonlyArray ? ReadonlyArray> + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; + +type KeysOfUnion = T extends T ? keyof T : never; +export type Exact = P extends Builtin ? P + : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; + +function longToNumber(int64: { toString(): string }): number { + const num = globalThis.Number(int64.toString()); + if (num > globalThis.Number.MAX_SAFE_INTEGER) { + throw new globalThis.Error("Value is larger than Number.MAX_SAFE_INTEGER"); + } + if (num < globalThis.Number.MIN_SAFE_INTEGER) { + throw new globalThis.Error("Value is smaller than Number.MIN_SAFE_INTEGER"); + } + return num; +} + +function isObject(value: any): boolean { + return typeof value === "object" && value !== null; +} + +function isSet(value: any): boolean { + return value !== null && value !== undefined; +} + +export interface MessageFns { + encode(message: T, writer?: BinaryWriter): BinaryWriter; + decode(input: BinaryReader | Uint8Array, length?: number): T; + fromJSON(object: any): T; + toJSON(message: T): unknown; + create, I>>(base?: I): T; + fromPartial, I>>(object: I): T; +} diff --git a/core/gui/src/app/common/type/proto/google/protobuf/descriptor.ts b/core/gui/src/app/common/type/proto/google/protobuf/descriptor.ts index fc4fce0c2cd..0a4648bd504 100644 --- a/core/gui/src/app/common/type/proto/google/protobuf/descriptor.ts +++ b/core/gui/src/app/common/type/proto/google/protobuf/descriptor.ts @@ -20,7 +20,7 @@ // Code generated by protoc-gen-ts_proto. DO NOT EDIT. // versions: // protoc-gen-ts_proto v2.2.0 -// protoc v5.28.3 +// protoc v5.29.3 // source: google/protobuf/descriptor.proto /* eslint-disable */ @@ -54,7 +54,7 @@ export enum Edition { EDITION_2024 = 1001, /** * EDITION_1_TEST_ONLY - Placeholder editions for testing feature resolution. These should not be - * used or relyed on outside of tests. + * used or relied on outside of tests. */ EDITION_1_TEST_ONLY = 1, EDITION_2_TEST_ONLY = 2, @@ -158,9 +158,13 @@ export interface FileDescriptorSet { /** Describes a complete .proto file. */ export interface FileDescriptorProto { /** file name, relative to root of source tree */ - name?: string | undefined; + name?: + | string + | undefined; /** e.g. "foo", "foo.bar", etc. */ - package?: string | undefined; + package?: + | string + | undefined; /** Names of files imported by this file. */ dependency: string[]; /** Indexes of the public imported files in the dependency list above. */ @@ -175,21 +179,27 @@ export interface FileDescriptorProto { enumType: EnumDescriptorProto[]; service: ServiceDescriptorProto[]; extension: FieldDescriptorProto[]; - options?: FileOptions | undefined; + options?: + | FileOptions + | undefined; /** * This field contains optional information about the original source code. * You may safely remove this entire field without harming runtime * functionality of the descriptors -- the information is needed only by * development tools. */ - sourceCodeInfo?: SourceCodeInfo | undefined; + sourceCodeInfo?: + | SourceCodeInfo + | undefined; /** * The syntax of the proto file. * The supported values are "proto2", "proto3", and "editions". * * If `edition` is present, this value must be "editions". */ - syntax?: string | undefined; + syntax?: + | string + | undefined; /** The edition of the proto file. */ edition?: Edition | undefined; } @@ -214,7 +224,9 @@ export interface DescriptorProto { export interface DescriptorProto_ExtensionRange { /** Inclusive. */ - start?: number | undefined; + start?: + | number + | undefined; /** Exclusive. */ end?: number | undefined; options?: ExtensionRangeOptions | undefined; @@ -227,7 +239,9 @@ export interface DescriptorProto_ExtensionRange { */ export interface DescriptorProto_ReservedRange { /** Inclusive. */ - start?: number | undefined; + start?: + | number + | undefined; /** Exclusive. */ end?: number | undefined; } @@ -242,7 +256,9 @@ export interface ExtensionRangeOptions { */ declaration: ExtensionRangeOptions_Declaration[]; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** * The verification state of the range. * TODO: flip the default to DECLARATION once all empty ranges @@ -288,24 +304,32 @@ export function extensionRangeOptions_VerificationStateToJSON(object: ExtensionR export interface ExtensionRangeOptions_Declaration { /** The extension number declared within the extension range. */ - number?: number | undefined; + number?: + | number + | undefined; /** * The fully-qualified name of the extension field. There must be a leading * dot in front of the full name. */ - fullName?: string | undefined; + fullName?: + | string + | undefined; /** * The fully-qualified type name of the extension field. Unlike * Metadata.type, Declaration.type must have a leading dot for messages * and enums. */ - type?: string | undefined; + type?: + | string + | undefined; /** * If true, indicates that the number is reserved in the extension range, * and any extension field with the number will fail to compile. Set this * when a declared extension field is deleted. */ - reserved?: boolean | undefined; + reserved?: + | boolean + | undefined; /** * If true, indicates that the extension must be defined as repeated. * Otherwise the extension must be defined as optional. @@ -317,12 +341,16 @@ export interface ExtensionRangeOptions_Declaration { export interface FieldDescriptorProto { name?: string | undefined; number?: number | undefined; - label?: FieldDescriptorProto_Label | undefined; + label?: + | FieldDescriptorProto_Label + | undefined; /** * If type_name is set, this need not be set. If both this and type_name * are set, this must be one of TYPE_ENUM, TYPE_MESSAGE or TYPE_GROUP. */ - type?: FieldDescriptorProto_Type | undefined; + type?: + | FieldDescriptorProto_Type + | undefined; /** * For message and enum types, this is the name of the type. If the name * starts with a '.', it is fully-qualified. Otherwise, C++-like scoping @@ -330,24 +358,32 @@ export interface FieldDescriptorProto { * message are searched, then within the parent, on up to the root * namespace). */ - typeName?: string | undefined; + typeName?: + | string + | undefined; /** * For extensions, this is the name of the type being extended. It is * resolved in the same manner as type_name. */ - extendee?: string | undefined; + extendee?: + | string + | undefined; /** * For numeric types, contains the original text representation of the value. * For booleans, "true" or "false". * For strings, contains the default text contents (not escaped in any way). * For bytes, contains the C escaped value. All bytes >= 128 are escaped. */ - defaultValue?: string | undefined; + defaultValue?: + | string + | undefined; /** * If set, gives the index of a oneof in the containing type's oneof_decl * list. This field is a member of that oneof. */ - oneofIndex?: number | undefined; + oneofIndex?: + | number + | undefined; /** * JSON name of this field. The value is set by protocol compiler. If the * user has set a "json_name" option on this field, that option's value @@ -355,7 +391,9 @@ export interface FieldDescriptorProto { * it to camelCase. */ jsonName?: string | undefined; - options?: FieldOptions | undefined; + options?: + | FieldOptions + | undefined; /** * If true, this is a proto3 "optional". When a proto3 field is optional, it * tracks presence regardless of field type. @@ -589,7 +627,9 @@ export interface OneofDescriptorProto { export interface EnumDescriptorProto { name?: string | undefined; value: EnumValueDescriptorProto[]; - options?: EnumOptions | undefined; + options?: + | EnumOptions + | undefined; /** * Range of reserved numeric values. Reserved numeric values may not be used * by enum values in the same enum declaration. Reserved ranges may not @@ -613,7 +653,9 @@ export interface EnumDescriptorProto { */ export interface EnumDescriptorProto_EnumReservedRange { /** Inclusive. */ - start?: number | undefined; + start?: + | number + | undefined; /** Inclusive. */ end?: number | undefined; } @@ -634,16 +676,22 @@ export interface ServiceDescriptorProto { /** Describes a method of a service. */ export interface MethodDescriptorProto { - name?: string | undefined; + name?: + | string + | undefined; /** * Input and output type names. These are resolved in the same way as * FieldDescriptorProto.type_name, but must refer to a message type. */ inputType?: string | undefined; outputType?: string | undefined; - options?: MethodOptions | undefined; + options?: + | MethodOptions + | undefined; /** Identifies if client streams multiple client messages */ - clientStreaming?: boolean | undefined; + clientStreaming?: + | boolean + | undefined; /** Identifies if server streams multiple server messages */ serverStreaming?: boolean | undefined; } @@ -655,7 +703,9 @@ export interface FileOptions { * inappropriate because proto packages do not normally start with backwards * domain names. */ - javaPackage?: string | undefined; + javaPackage?: + | string + | undefined; /** * Controls the name of the wrapper Java class generated for the .proto file. * That class will always contain the .proto file's getDescriptor() method as @@ -663,7 +713,9 @@ export interface FileOptions { * If java_multiple_files is disabled, then all the other classes from the * .proto file will be nested inside the single wrapper outer class. */ - javaOuterClassname?: string | undefined; + javaOuterClassname?: + | string + | undefined; /** * If enabled, then the Java code generator will generate a separate .java * file for each top-level message, enum, and service defined in the .proto @@ -672,13 +724,17 @@ export interface FileOptions { * generated to contain the file's getDescriptor() method as well as any * top-level extensions defined in the file. */ - javaMultipleFiles?: boolean | undefined; + javaMultipleFiles?: + | boolean + | undefined; /** * This option does nothing. * * @deprecated */ - javaGenerateEqualsAndHash?: boolean | undefined; + javaGenerateEqualsAndHash?: + | boolean + | undefined; /** * A proto2 file can set this to true to opt in to UTF-8 checking for Java, * which will throw an exception if invalid UTF-8 is parsed from the wire or @@ -692,7 +748,9 @@ export interface FileOptions { * checks. */ javaStringCheckUtf8?: boolean | undefined; - optimizeFor?: FileOptions_OptimizeMode | undefined; + optimizeFor?: + | FileOptions_OptimizeMode + | undefined; /** * Sets the Go package where structs generated from this .proto will be * placed. If omitted, the Go package will be derived from the following: @@ -700,7 +758,9 @@ export interface FileOptions { * - Otherwise, the package statement in the .proto file, if present. * - Otherwise, the basename of the .proto file, without extension. */ - goPackage?: string | undefined; + goPackage?: + | string + | undefined; /** * Should generic services be generated in each language? "Generic" services * are not specific to any particular RPC system. They are generated by the @@ -715,58 +775,80 @@ export interface FileOptions { */ ccGenericServices?: boolean | undefined; javaGenericServices?: boolean | undefined; - pyGenericServices?: boolean | undefined; + pyGenericServices?: + | boolean + | undefined; /** * Is this file deprecated? * Depending on the target platform, this can emit Deprecated annotations * for everything in the file, or it will be completely ignored; in the very * least, this is a formalization for deprecating files. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** * Enables the use of arenas for the proto messages in this file. This applies * only to generated classes for C++. */ - ccEnableArenas?: boolean | undefined; + ccEnableArenas?: + | boolean + | undefined; /** * Sets the objective c class prefix which is prepended to all objective c * generated classes from this .proto. There is no default. */ - objcClassPrefix?: string | undefined; + objcClassPrefix?: + | string + | undefined; /** Namespace for generated classes; defaults to the package. */ - csharpNamespace?: string | undefined; + csharpNamespace?: + | string + | undefined; /** * By default Swift generators will take the proto package and CamelCase it * replacing '.' with underscore and use that to prefix the types/symbols * defined. When this options is provided, they will use this value instead * to prefix the types/symbols defined. */ - swiftPrefix?: string | undefined; + swiftPrefix?: + | string + | undefined; /** * Sets the php class prefix which is prepended to all php generated classes * from this .proto. Default is empty. */ - phpClassPrefix?: string | undefined; + phpClassPrefix?: + | string + | undefined; /** * Use this option to change the namespace of php generated classes. Default * is empty. When this option is empty, the package name will be used for * determining the namespace. */ - phpNamespace?: string | undefined; + phpNamespace?: + | string + | undefined; /** * Use this option to change the namespace of php generated metadata classes. * Default is empty. When this option is empty, the proto file name will be * used for determining the namespace. */ - phpMetadataNamespace?: string | undefined; + phpMetadataNamespace?: + | string + | undefined; /** * Use this option to change the package of ruby generated classes. Default * is empty. When this option is not set, the package name will be used for * determining the ruby package. */ - rubyPackage?: string | undefined; + rubyPackage?: + | string + | undefined; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** * The parser stores options it doesn't recognize here. * See the documentation for the "Options" section above. @@ -838,20 +920,26 @@ export interface MessageOptions { * Because this is an option, the above two restrictions are not enforced by * the protocol compiler. */ - messageSetWireFormat?: boolean | undefined; + messageSetWireFormat?: + | boolean + | undefined; /** * Disables the generation of the standard "descriptor()" accessor, which can * conflict with a field of the same name. This is meant to make migration * from proto1 easier; new code should avoid fields named "descriptor". */ - noStandardDescriptorAccessor?: boolean | undefined; + noStandardDescriptorAccessor?: + | boolean + | undefined; /** * Is this message deprecated? * Depending on the target platform, this can emit Deprecated annotations * for the message, or it will be completely ignored; in the very least, * this is a formalization for deprecating messages. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** * Whether the message is an automatically generated map entry type for the * maps field. @@ -875,7 +963,9 @@ export interface MessageOptions { * instead. The option should only be implicitly set by the proto compiler * parser. */ - mapEntry?: boolean | undefined; + mapEntry?: + | boolean + | undefined; /** * Enable the legacy handling of JSON field name conflicts. This lowercases * and strips underscored from the fields before comparison in proto3 only. @@ -890,9 +980,13 @@ export interface MessageOptions { * * @deprecated */ - deprecatedLegacyJsonFieldConflicts?: boolean | undefined; + deprecatedLegacyJsonFieldConflicts?: + | boolean + | undefined; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -907,7 +1001,9 @@ export interface FieldOptions { * type "bytes" in the open source release. * TODO: make ctype actually deprecated. */ - ctype?: FieldOptions_CType | undefined; + ctype?: + | FieldOptions_CType + | undefined; /** * The packed option can be enabled for repeated primitive fields to enable * a more efficient representation on the wire. Rather than repeatedly @@ -917,7 +1013,9 @@ export interface FieldOptions { * Editions, but the `repeated_field_encoding` feature can be used to control * the behavior. */ - packed?: boolean | undefined; + packed?: + | boolean + | undefined; /** * The jstype option determines the JavaScript type used for values of the * field. The option is permitted only for 64 bit integral and fixed types @@ -931,7 +1029,9 @@ export interface FieldOptions { * This option is an enum to permit additional types to be added, e.g. * goog.math.Integer. */ - jstype?: FieldOptions_JSType | undefined; + jstype?: + | FieldOptions_JSType + | undefined; /** * Should this field be parsed lazily? Lazy applies only to message-type * fields. It means that when the outer message is initially parsed, the @@ -956,22 +1056,30 @@ export interface FieldOptions { * fields. Failed verification would result in parsing failure (except when * uninitialized messages are acceptable). */ - lazy?: boolean | undefined; + lazy?: + | boolean + | undefined; /** * unverified_lazy does no correctness checks on the byte stream. This should * only be used where lazy with verification is prohibitive for performance * reasons. */ - unverifiedLazy?: boolean | undefined; + unverifiedLazy?: + | boolean + | undefined; /** * Is this field deprecated? * Depending on the target platform, this can emit Deprecated annotations * for accessors, or it will be completely ignored; in the very least, this * is a formalization for deprecating fields. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** For Google-internal migration only. Do not use. */ - weak?: boolean | undefined; + weak?: + | boolean + | undefined; /** * Indicate that the field value should not be printed out when using debug * formats, e.g. when the field contains sensitive credentials. @@ -982,7 +1090,9 @@ export interface FieldOptions { editionDefaults: FieldOptions_EditionDefault[]; /** Any features defined in the specific edition. */ features?: FeatureSet | undefined; - featureSupport?: FieldOptions_FeatureSupport | undefined; + featureSupport?: + | FieldOptions_FeatureSupport + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1077,11 +1187,7 @@ export function fieldOptions_JSTypeToJSON(object: FieldOptions_JSType): string { } } -/** - * If set to RETENTION_SOURCE, the option will be omitted from the binary. - * Note: as of January 2023, support for this is in progress and does not yet - * have an effect (b/264593489). - */ +/** If set to RETENTION_SOURCE, the option will be omitted from the binary. */ export enum FieldOptions_OptionRetention { RETENTION_UNKNOWN = 0, RETENTION_RUNTIME = 1, @@ -1124,8 +1230,7 @@ export function fieldOptions_OptionRetentionToJSON(object: FieldOptions_OptionRe /** * This indicates the types of entities that the field may apply to when used * as an option. If it is unset, then the field may be freely used as an - * option on any kind of entity. Note: as of January 2023, support for this is - * in progress and does not yet have an effect (b/264593489). + * option on any kind of entity. */ export enum FieldOptions_OptionTargetType { TARGET_TYPE_UNKNOWN = 0, @@ -1209,7 +1314,9 @@ export function fieldOptions_OptionTargetTypeToJSON(object: FieldOptions_OptionT } export interface FieldOptions_EditionDefault { - edition?: Edition | undefined; + edition?: + | Edition + | undefined; /** Textproto value. */ value?: string | undefined; } @@ -1221,17 +1328,23 @@ export interface FieldOptions_FeatureSupport { * earlier than this one, the default assigned to EDITION_LEGACY will be * used, and proto files will not be able to override it. */ - editionIntroduced?: Edition | undefined; + editionIntroduced?: + | Edition + | undefined; /** * The edition this feature becomes deprecated in. Using this after this * edition may trigger warnings. */ - editionDeprecated?: Edition | undefined; + editionDeprecated?: + | Edition + | undefined; /** * The deprecation warning text if this feature is used after the edition it * was marked deprecated in. */ - deprecationWarning?: string | undefined; + deprecationWarning?: + | string + | undefined; /** * The edition this feature is no longer available in. In editions after * this one, the last default assigned will be used, and proto files will @@ -1242,7 +1355,9 @@ export interface FieldOptions_FeatureSupport { export interface OneofOptions { /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1252,14 +1367,18 @@ export interface EnumOptions { * Set this option to true to allow mapping different tag names to the same * value. */ - allowAlias?: boolean | undefined; + allowAlias?: + | boolean + | undefined; /** * Is this enum deprecated? * Depending on the target platform, this can emit Deprecated annotations * for the enum, or it will be completely ignored; in the very least, this * is a formalization for deprecating enums. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** * Enable the legacy handling of JSON field name conflicts. This lowercases * and strips underscored from the fields before comparison in proto3 only. @@ -1270,9 +1389,13 @@ export interface EnumOptions { * * @deprecated */ - deprecatedLegacyJsonFieldConflicts?: boolean | undefined; + deprecatedLegacyJsonFieldConflicts?: + | boolean + | undefined; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1284,31 +1407,43 @@ export interface EnumValueOptions { * for the enum value, or it will be completely ignored; in the very least, * this is a formalization for deprecating enum values. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** * Indicate that fields annotated with this enum value should not be printed * out when using debug formats, e.g. when the field contains sensitive * credentials. */ - debugRedact?: boolean | undefined; + debugRedact?: + | boolean + | undefined; /** Information about the support window of a feature value. */ - featureSupport?: FieldOptions_FeatureSupport | undefined; + featureSupport?: + | FieldOptions_FeatureSupport + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } export interface ServiceOptions { /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** * Is this service deprecated? * Depending on the target platform, this can emit Deprecated annotations * for the service, or it will be completely ignored; in the very least, * this is a formalization for deprecating services. */ - deprecated?: boolean | undefined; + deprecated?: + | boolean + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1321,9 +1456,13 @@ export interface MethodOptions { * this is a formalization for deprecating methods. */ deprecated?: boolean | undefined; - idempotencyLevel?: MethodOptions_IdempotencyLevel | undefined; + idempotencyLevel?: + | MethodOptions_IdempotencyLevel + | undefined; /** Any features defined in the specific edition. */ - features?: FeatureSet | undefined; + features?: + | FeatureSet + | undefined; /** The parser stores options it doesn't recognize here. See above. */ uninterpretedOption: UninterpretedOption[]; } @@ -1677,7 +1816,9 @@ export interface FeatureSetDefaults { * The minimum supported edition (inclusive) when this was constructed. * Editions before this will not have defaults. */ - minimumEdition?: Edition | undefined; + minimumEdition?: + | Edition + | undefined; /** * The maximum known edition (inclusive) when this was constructed. Editions * after this will not have reliable defaults. @@ -1692,9 +1833,13 @@ export interface FeatureSetDefaults { * be used. This field must be in strict ascending order by edition. */ export interface FeatureSetDefaults_FeatureSetEditionDefault { - edition?: Edition | undefined; + edition?: + | Edition + | undefined; /** Defaults of features that can be overridden in this edition. */ - overridableFeatures?: FeatureSet | undefined; + overridableFeatures?: + | FeatureSet + | undefined; /** Defaults of features that can't be overridden in this edition. */ fixedFeatures?: FeatureSet | undefined; } @@ -1861,12 +2006,16 @@ export interface GeneratedCodeInfo_Annotation { */ path: number[]; /** Identifies the filesystem path to the original source .proto. */ - sourceFile?: string | undefined; + sourceFile?: + | string + | undefined; /** * Identifies the starting offset in bytes in the generated code * that relates to the identified object. */ - begin?: number | undefined; + begin?: + | number + | undefined; /** * Identifies the ending offset in bytes in the generated code that * relates to the identified object. The end offset should be one past @@ -1966,7 +2115,7 @@ export const FileDescriptorSet: MessageFns = { toJSON(message: FileDescriptorSet): unknown { const obj: any = {}; if (message.file?.length) { - obj.file = message.file.map(e => FileDescriptorProto.toJSON(e)); + obj.file = message.file.map((e) => FileDescriptorProto.toJSON(e)); } return obj; }, @@ -1976,7 +2125,7 @@ export const FileDescriptorSet: MessageFns = { }, fromPartial, I>>(object: I): FileDescriptorSet { const message = createBaseFileDescriptorSet(); - message.file = object.file?.map(e => FileDescriptorProto.fromPartial(e)) || []; + message.file = object.file?.map((e) => FileDescriptorProto.fromPartial(e)) || []; return message; }, }; @@ -2218,22 +2367,22 @@ export const FileDescriptorProto: MessageFns = { obj.dependency = message.dependency; } if (message.publicDependency?.length) { - obj.publicDependency = message.publicDependency.map(e => Math.round(e)); + obj.publicDependency = message.publicDependency.map((e) => Math.round(e)); } if (message.weakDependency?.length) { - obj.weakDependency = message.weakDependency.map(e => Math.round(e)); + obj.weakDependency = message.weakDependency.map((e) => Math.round(e)); } if (message.messageType?.length) { - obj.messageType = message.messageType.map(e => DescriptorProto.toJSON(e)); + obj.messageType = message.messageType.map((e) => DescriptorProto.toJSON(e)); } if (message.enumType?.length) { - obj.enumType = message.enumType.map(e => EnumDescriptorProto.toJSON(e)); + obj.enumType = message.enumType.map((e) => EnumDescriptorProto.toJSON(e)); } if (message.service?.length) { - obj.service = message.service.map(e => ServiceDescriptorProto.toJSON(e)); + obj.service = message.service.map((e) => ServiceDescriptorProto.toJSON(e)); } if (message.extension?.length) { - obj.extension = message.extension.map(e => FieldDescriptorProto.toJSON(e)); + obj.extension = message.extension.map((e) => FieldDescriptorProto.toJSON(e)); } if (message.options !== undefined) { obj.options = FileOptions.toJSON(message.options); @@ -2257,19 +2406,19 @@ export const FileDescriptorProto: MessageFns = { const message = createBaseFileDescriptorProto(); message.name = object.name ?? ""; message.package = object.package ?? ""; - message.dependency = object.dependency?.map(e => e) || []; - message.publicDependency = object.publicDependency?.map(e => e) || []; - message.weakDependency = object.weakDependency?.map(e => e) || []; - message.messageType = object.messageType?.map(e => DescriptorProto.fromPartial(e)) || []; - message.enumType = object.enumType?.map(e => EnumDescriptorProto.fromPartial(e)) || []; - message.service = object.service?.map(e => ServiceDescriptorProto.fromPartial(e)) || []; - message.extension = object.extension?.map(e => FieldDescriptorProto.fromPartial(e)) || []; - message.options = - object.options !== undefined && object.options !== null ? FileOptions.fromPartial(object.options) : undefined; - message.sourceCodeInfo = - object.sourceCodeInfo !== undefined && object.sourceCodeInfo !== null - ? SourceCodeInfo.fromPartial(object.sourceCodeInfo) - : undefined; + message.dependency = object.dependency?.map((e) => e) || []; + message.publicDependency = object.publicDependency?.map((e) => e) || []; + message.weakDependency = object.weakDependency?.map((e) => e) || []; + message.messageType = object.messageType?.map((e) => DescriptorProto.fromPartial(e)) || []; + message.enumType = object.enumType?.map((e) => EnumDescriptorProto.fromPartial(e)) || []; + message.service = object.service?.map((e) => ServiceDescriptorProto.fromPartial(e)) || []; + message.extension = object.extension?.map((e) => FieldDescriptorProto.fromPartial(e)) || []; + message.options = (object.options !== undefined && object.options !== null) + ? FileOptions.fromPartial(object.options) + : undefined; + message.sourceCodeInfo = (object.sourceCodeInfo !== undefined && object.sourceCodeInfo !== null) + ? SourceCodeInfo.fromPartial(object.sourceCodeInfo) + : undefined; message.syntax = object.syntax ?? ""; message.edition = object.edition ?? 0; return message; @@ -2449,28 +2598,28 @@ export const DescriptorProto: MessageFns = { obj.name = message.name; } if (message.field?.length) { - obj.field = message.field.map(e => FieldDescriptorProto.toJSON(e)); + obj.field = message.field.map((e) => FieldDescriptorProto.toJSON(e)); } if (message.extension?.length) { - obj.extension = message.extension.map(e => FieldDescriptorProto.toJSON(e)); + obj.extension = message.extension.map((e) => FieldDescriptorProto.toJSON(e)); } if (message.nestedType?.length) { - obj.nestedType = message.nestedType.map(e => DescriptorProto.toJSON(e)); + obj.nestedType = message.nestedType.map((e) => DescriptorProto.toJSON(e)); } if (message.enumType?.length) { - obj.enumType = message.enumType.map(e => EnumDescriptorProto.toJSON(e)); + obj.enumType = message.enumType.map((e) => EnumDescriptorProto.toJSON(e)); } if (message.extensionRange?.length) { - obj.extensionRange = message.extensionRange.map(e => DescriptorProto_ExtensionRange.toJSON(e)); + obj.extensionRange = message.extensionRange.map((e) => DescriptorProto_ExtensionRange.toJSON(e)); } if (message.oneofDecl?.length) { - obj.oneofDecl = message.oneofDecl.map(e => OneofDescriptorProto.toJSON(e)); + obj.oneofDecl = message.oneofDecl.map((e) => OneofDescriptorProto.toJSON(e)); } if (message.options !== undefined) { obj.options = MessageOptions.toJSON(message.options); } if (message.reservedRange?.length) { - obj.reservedRange = message.reservedRange.map(e => DescriptorProto_ReservedRange.toJSON(e)); + obj.reservedRange = message.reservedRange.map((e) => DescriptorProto_ReservedRange.toJSON(e)); } if (message.reservedName?.length) { obj.reservedName = message.reservedName; @@ -2484,16 +2633,17 @@ export const DescriptorProto: MessageFns = { fromPartial, I>>(object: I): DescriptorProto { const message = createBaseDescriptorProto(); message.name = object.name ?? ""; - message.field = object.field?.map(e => FieldDescriptorProto.fromPartial(e)) || []; - message.extension = object.extension?.map(e => FieldDescriptorProto.fromPartial(e)) || []; - message.nestedType = object.nestedType?.map(e => DescriptorProto.fromPartial(e)) || []; - message.enumType = object.enumType?.map(e => EnumDescriptorProto.fromPartial(e)) || []; - message.extensionRange = object.extensionRange?.map(e => DescriptorProto_ExtensionRange.fromPartial(e)) || []; - message.oneofDecl = object.oneofDecl?.map(e => OneofDescriptorProto.fromPartial(e)) || []; - message.options = - object.options !== undefined && object.options !== null ? MessageOptions.fromPartial(object.options) : undefined; - message.reservedRange = object.reservedRange?.map(e => DescriptorProto_ReservedRange.fromPartial(e)) || []; - message.reservedName = object.reservedName?.map(e => e) || []; + message.field = object.field?.map((e) => FieldDescriptorProto.fromPartial(e)) || []; + message.extension = object.extension?.map((e) => FieldDescriptorProto.fromPartial(e)) || []; + message.nestedType = object.nestedType?.map((e) => DescriptorProto.fromPartial(e)) || []; + message.enumType = object.enumType?.map((e) => EnumDescriptorProto.fromPartial(e)) || []; + message.extensionRange = object.extensionRange?.map((e) => DescriptorProto_ExtensionRange.fromPartial(e)) || []; + message.oneofDecl = object.oneofDecl?.map((e) => OneofDescriptorProto.fromPartial(e)) || []; + message.options = (object.options !== undefined && object.options !== null) + ? MessageOptions.fromPartial(object.options) + : undefined; + message.reservedRange = object.reservedRange?.map((e) => DescriptorProto_ReservedRange.fromPartial(e)) || []; + message.reservedName = object.reservedName?.map((e) => e) || []; return message; }, }; @@ -2579,15 +2729,14 @@ export const DescriptorProto_ExtensionRange: MessageFns, I>>( - object: I + object: I, ): DescriptorProto_ExtensionRange { const message = createBaseDescriptorProto_ExtensionRange(); message.start = object.start ?? 0; message.end = object.end ?? 0; - message.options = - object.options !== undefined && object.options !== null - ? ExtensionRangeOptions.fromPartial(object.options) - : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? ExtensionRangeOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -2659,7 +2808,7 @@ export const DescriptorProto_ReservedRange: MessageFns, I>>( - object: I + object: I, ): DescriptorProto_ReservedRange { const message = createBaseDescriptorProto_ReservedRange(); message.start = object.start ?? 0; @@ -2751,10 +2900,10 @@ export const ExtensionRangeOptions: MessageFns = { toJSON(message: ExtensionRangeOptions): unknown { const obj: any = {}; if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } if (message.declaration?.length) { - obj.declaration = message.declaration.map(e => ExtensionRangeOptions_Declaration.toJSON(e)); + obj.declaration = message.declaration.map((e) => ExtensionRangeOptions_Declaration.toJSON(e)); } if (message.features !== undefined) { obj.features = FeatureSet.toJSON(message.features); @@ -2770,10 +2919,11 @@ export const ExtensionRangeOptions: MessageFns = { }, fromPartial, I>>(object: I): ExtensionRangeOptions { const message = createBaseExtensionRangeOptions(); - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; - message.declaration = object.declaration?.map(e => ExtensionRangeOptions_Declaration.fromPartial(e)) || []; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; + message.declaration = object.declaration?.map((e) => ExtensionRangeOptions_Declaration.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.verification = object.verification ?? 1; return message; }, @@ -2885,12 +3035,12 @@ export const ExtensionRangeOptions_Declaration: MessageFns, I>>( - base?: I + base?: I, ): ExtensionRangeOptions_Declaration { return ExtensionRangeOptions_Declaration.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): ExtensionRangeOptions_Declaration { const message = createBaseExtensionRangeOptions_Declaration(); message.number = object.number ?? 0; @@ -3117,8 +3267,9 @@ export const FieldDescriptorProto: MessageFns = { message.defaultValue = object.defaultValue ?? ""; message.oneofIndex = object.oneofIndex ?? 0; message.jsonName = object.jsonName ?? ""; - message.options = - object.options !== undefined && object.options !== null ? FieldOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? FieldOptions.fromPartial(object.options) + : undefined; message.proto3Optional = object.proto3Optional ?? false; return message; }, @@ -3193,8 +3344,9 @@ export const OneofDescriptorProto: MessageFns = { fromPartial, I>>(object: I): OneofDescriptorProto { const message = createBaseOneofDescriptorProto(); message.name = object.name ?? ""; - message.options = - object.options !== undefined && object.options !== null ? OneofOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? OneofOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -3296,13 +3448,13 @@ export const EnumDescriptorProto: MessageFns = { obj.name = message.name; } if (message.value?.length) { - obj.value = message.value.map(e => EnumValueDescriptorProto.toJSON(e)); + obj.value = message.value.map((e) => EnumValueDescriptorProto.toJSON(e)); } if (message.options !== undefined) { obj.options = EnumOptions.toJSON(message.options); } if (message.reservedRange?.length) { - obj.reservedRange = message.reservedRange.map(e => EnumDescriptorProto_EnumReservedRange.toJSON(e)); + obj.reservedRange = message.reservedRange.map((e) => EnumDescriptorProto_EnumReservedRange.toJSON(e)); } if (message.reservedName?.length) { obj.reservedName = message.reservedName; @@ -3316,11 +3468,13 @@ export const EnumDescriptorProto: MessageFns = { fromPartial, I>>(object: I): EnumDescriptorProto { const message = createBaseEnumDescriptorProto(); message.name = object.name ?? ""; - message.value = object.value?.map(e => EnumValueDescriptorProto.fromPartial(e)) || []; - message.options = - object.options !== undefined && object.options !== null ? EnumOptions.fromPartial(object.options) : undefined; - message.reservedRange = object.reservedRange?.map(e => EnumDescriptorProto_EnumReservedRange.fromPartial(e)) || []; - message.reservedName = object.reservedName?.map(e => e) || []; + message.value = object.value?.map((e) => EnumValueDescriptorProto.fromPartial(e)) || []; + message.options = (object.options !== undefined && object.options !== null) + ? EnumOptions.fromPartial(object.options) + : undefined; + message.reservedRange = object.reservedRange?.map((e) => EnumDescriptorProto_EnumReservedRange.fromPartial(e)) || + []; + message.reservedName = object.reservedName?.map((e) => e) || []; return message; }, }; @@ -3389,12 +3543,12 @@ export const EnumDescriptorProto_EnumReservedRange: MessageFns, I>>( - base?: I + base?: I, ): EnumDescriptorProto_EnumReservedRange { return EnumDescriptorProto_EnumReservedRange.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): EnumDescriptorProto_EnumReservedRange { const message = createBaseEnumDescriptorProto_EnumReservedRange(); message.start = object.start ?? 0; @@ -3487,10 +3641,9 @@ export const EnumValueDescriptorProto: MessageFns = { const message = createBaseEnumValueDescriptorProto(); message.name = object.name ?? ""; message.number = object.number ?? 0; - message.options = - object.options !== undefined && object.options !== null - ? EnumValueOptions.fromPartial(object.options) - : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? EnumValueOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -3566,7 +3719,7 @@ export const ServiceDescriptorProto: MessageFns = { obj.name = message.name; } if (message.method?.length) { - obj.method = message.method.map(e => MethodDescriptorProto.toJSON(e)); + obj.method = message.method.map((e) => MethodDescriptorProto.toJSON(e)); } if (message.options !== undefined) { obj.options = ServiceOptions.toJSON(message.options); @@ -3580,9 +3733,10 @@ export const ServiceDescriptorProto: MessageFns = { fromPartial, I>>(object: I): ServiceDescriptorProto { const message = createBaseServiceDescriptorProto(); message.name = object.name ?? ""; - message.method = object.method?.map(e => MethodDescriptorProto.fromPartial(e)) || []; - message.options = - object.options !== undefined && object.options !== null ? ServiceOptions.fromPartial(object.options) : undefined; + message.method = object.method?.map((e) => MethodDescriptorProto.fromPartial(e)) || []; + message.options = (object.options !== undefined && object.options !== null) + ? ServiceOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -3721,8 +3875,9 @@ export const MethodDescriptorProto: MessageFns = { message.name = object.name ?? ""; message.inputType = object.inputType ?? ""; message.outputType = object.outputType ?? ""; - message.options = - object.options !== undefined && object.options !== null ? MethodOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? MethodOptions.fromPartial(object.options) + : undefined; message.clientStreaming = object.clientStreaming ?? false; message.serverStreaming = object.serverStreaming ?? false; return message; @@ -4079,7 +4234,7 @@ export const FileOptions: MessageFns = { obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -4108,9 +4263,10 @@ export const FileOptions: MessageFns = { message.phpNamespace = object.phpNamespace ?? ""; message.phpMetadataNamespace = object.phpMetadataNamespace ?? ""; message.rubyPackage = object.rubyPackage ?? ""; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -4142,8 +4298,7 @@ export const MessageOptions: MessageFns = { writer.uint32(56).bool(message.mapEntry); } if ( - message.deprecatedLegacyJsonFieldConflicts !== undefined && - message.deprecatedLegacyJsonFieldConflicts !== false + message.deprecatedLegacyJsonFieldConflicts !== undefined && message.deprecatedLegacyJsonFieldConflicts !== false ) { writer.uint32(88).bool(message.deprecatedLegacyJsonFieldConflicts); } @@ -4256,8 +4411,7 @@ export const MessageOptions: MessageFns = { obj.mapEntry = message.mapEntry; } if ( - message.deprecatedLegacyJsonFieldConflicts !== undefined && - message.deprecatedLegacyJsonFieldConflicts !== false + message.deprecatedLegacyJsonFieldConflicts !== undefined && message.deprecatedLegacyJsonFieldConflicts !== false ) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } @@ -4265,7 +4419,7 @@ export const MessageOptions: MessageFns = { obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -4280,9 +4434,10 @@ export const MessageOptions: MessageFns = { message.deprecated = object.deprecated ?? false; message.mapEntry = object.mapEntry ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -4536,10 +4691,10 @@ export const FieldOptions: MessageFns = { obj.retention = fieldOptions_OptionRetentionToJSON(message.retention); } if (message.targets?.length) { - obj.targets = message.targets.map(e => fieldOptions_OptionTargetTypeToJSON(e)); + obj.targets = message.targets.map((e) => fieldOptions_OptionTargetTypeToJSON(e)); } if (message.editionDefaults?.length) { - obj.editionDefaults = message.editionDefaults.map(e => FieldOptions_EditionDefault.toJSON(e)); + obj.editionDefaults = message.editionDefaults.map((e) => FieldOptions_EditionDefault.toJSON(e)); } if (message.features !== undefined) { obj.features = FeatureSet.toJSON(message.features); @@ -4548,7 +4703,7 @@ export const FieldOptions: MessageFns = { obj.featureSupport = FieldOptions_FeatureSupport.toJSON(message.featureSupport); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -4567,15 +4722,15 @@ export const FieldOptions: MessageFns = { message.weak = object.weak ?? false; message.debugRedact = object.debugRedact ?? false; message.retention = object.retention ?? 0; - message.targets = object.targets?.map(e => e) || []; - message.editionDefaults = object.editionDefaults?.map(e => FieldOptions_EditionDefault.fromPartial(e)) || []; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.featureSupport = - object.featureSupport !== undefined && object.featureSupport !== null - ? FieldOptions_FeatureSupport.fromPartial(object.featureSupport) - : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.targets = object.targets?.map((e) => e) || []; + message.editionDefaults = object.editionDefaults?.map((e) => FieldOptions_EditionDefault.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.featureSupport = (object.featureSupport !== undefined && object.featureSupport !== null) + ? FieldOptions_FeatureSupport.fromPartial(object.featureSupport) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -4818,7 +4973,7 @@ export const OneofOptions: MessageFns = { obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -4828,9 +4983,10 @@ export const OneofOptions: MessageFns = { }, fromPartial, I>>(object: I): OneofOptions { const message = createBaseOneofOptions(); - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -4854,8 +5010,7 @@ export const EnumOptions: MessageFns = { writer.uint32(24).bool(message.deprecated); } if ( - message.deprecatedLegacyJsonFieldConflicts !== undefined && - message.deprecatedLegacyJsonFieldConflicts !== false + message.deprecatedLegacyJsonFieldConflicts !== undefined && message.deprecatedLegacyJsonFieldConflicts !== false ) { writer.uint32(48).bool(message.deprecatedLegacyJsonFieldConflicts); } @@ -4942,8 +5097,7 @@ export const EnumOptions: MessageFns = { obj.deprecated = message.deprecated; } if ( - message.deprecatedLegacyJsonFieldConflicts !== undefined && - message.deprecatedLegacyJsonFieldConflicts !== false + message.deprecatedLegacyJsonFieldConflicts !== undefined && message.deprecatedLegacyJsonFieldConflicts !== false ) { obj.deprecatedLegacyJsonFieldConflicts = message.deprecatedLegacyJsonFieldConflicts; } @@ -4951,7 +5105,7 @@ export const EnumOptions: MessageFns = { obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -4964,9 +5118,10 @@ export const EnumOptions: MessageFns = { message.allowAlias = object.allowAlias ?? false; message.deprecated = object.deprecated ?? false; message.deprecatedLegacyJsonFieldConflicts = object.deprecatedLegacyJsonFieldConflicts ?? false; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -5081,7 +5236,7 @@ export const EnumValueOptions: MessageFns = { obj.featureSupport = FieldOptions_FeatureSupport.toJSON(message.featureSupport); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -5092,14 +5247,14 @@ export const EnumValueOptions: MessageFns = { fromPartial, I>>(object: I): EnumValueOptions { const message = createBaseEnumValueOptions(); message.deprecated = object.deprecated ?? false; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.debugRedact = object.debugRedact ?? false; - message.featureSupport = - object.featureSupport !== undefined && object.featureSupport !== null - ? FieldOptions_FeatureSupport.fromPartial(object.featureSupport) - : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.featureSupport = (object.featureSupport !== undefined && object.featureSupport !== null) + ? FieldOptions_FeatureSupport.fromPartial(object.featureSupport) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -5178,7 +5333,7 @@ export const ServiceOptions: MessageFns = { obj.deprecated = message.deprecated; } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -5188,10 +5343,11 @@ export const ServiceOptions: MessageFns = { }, fromPartial, I>>(object: I): ServiceOptions { const message = createBaseServiceOptions(); - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; message.deprecated = object.deprecated ?? false; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -5286,7 +5442,7 @@ export const MethodOptions: MessageFns = { obj.features = FeatureSet.toJSON(message.features); } if (message.uninterpretedOption?.length) { - obj.uninterpretedOption = message.uninterpretedOption.map(e => UninterpretedOption.toJSON(e)); + obj.uninterpretedOption = message.uninterpretedOption.map((e) => UninterpretedOption.toJSON(e)); } return obj; }, @@ -5298,9 +5454,10 @@ export const MethodOptions: MessageFns = { const message = createBaseMethodOptions(); message.deprecated = object.deprecated ?? false; message.idempotencyLevel = object.idempotencyLevel ?? 0; - message.features = - object.features !== undefined && object.features !== null ? FeatureSet.fromPartial(object.features) : undefined; - message.uninterpretedOption = object.uninterpretedOption?.map(e => UninterpretedOption.fromPartial(e)) || []; + message.features = (object.features !== undefined && object.features !== null) + ? FeatureSet.fromPartial(object.features) + : undefined; + message.uninterpretedOption = object.uninterpretedOption?.map((e) => UninterpretedOption.fromPartial(e)) || []; return message; }, }; @@ -5425,7 +5582,7 @@ export const UninterpretedOption: MessageFns = { toJSON(message: UninterpretedOption): unknown { const obj: any = {}; if (message.name?.length) { - obj.name = message.name.map(e => UninterpretedOption_NamePart.toJSON(e)); + obj.name = message.name.map((e) => UninterpretedOption_NamePart.toJSON(e)); } if (message.identifierValue !== undefined && message.identifierValue !== "") { obj.identifierValue = message.identifierValue; @@ -5453,7 +5610,7 @@ export const UninterpretedOption: MessageFns = { }, fromPartial, I>>(object: I): UninterpretedOption { const message = createBaseUninterpretedOption(); - message.name = object.name?.map(e => UninterpretedOption_NamePart.fromPartial(e)) || []; + message.name = object.name?.map((e) => UninterpretedOption_NamePart.fromPartial(e)) || []; message.identifierValue = object.identifierValue ?? ""; message.positiveIntValue = object.positiveIntValue ?? 0; message.negativeIntValue = object.negativeIntValue ?? 0; @@ -5749,7 +5906,7 @@ export const FeatureSetDefaults: MessageFns = { toJSON(message: FeatureSetDefaults): unknown { const obj: any = {}; if (message.defaults?.length) { - obj.defaults = message.defaults.map(e => FeatureSetDefaults_FeatureSetEditionDefault.toJSON(e)); + obj.defaults = message.defaults.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.toJSON(e)); } if (message.minimumEdition !== undefined && message.minimumEdition !== 0) { obj.minimumEdition = editionToJSON(message.minimumEdition); @@ -5765,7 +5922,7 @@ export const FeatureSetDefaults: MessageFns = { }, fromPartial, I>>(object: I): FeatureSetDefaults { const message = createBaseFeatureSetDefaults(); - message.defaults = object.defaults?.map(e => FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(e)) || []; + message.defaults = object.defaults?.map((e) => FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(e)) || []; message.minimumEdition = object.minimumEdition ?? 0; message.maximumEdition = object.maximumEdition ?? 0; return message; @@ -5779,7 +5936,7 @@ function createBaseFeatureSetDefaults_FeatureSetEditionDefault(): FeatureSetDefa export const FeatureSetDefaults_FeatureSetEditionDefault: MessageFns = { encode( message: FeatureSetDefaults_FeatureSetEditionDefault, - writer: BinaryWriter = new BinaryWriter() + writer: BinaryWriter = new BinaryWriter(), ): BinaryWriter { if (message.edition !== undefined && message.edition !== 0) { writer.uint32(24).int32(message.edition); @@ -5855,23 +6012,21 @@ export const FeatureSetDefaults_FeatureSetEditionDefault: MessageFns, I>>( - base?: I + base?: I, ): FeatureSetDefaults_FeatureSetEditionDefault { return FeatureSetDefaults_FeatureSetEditionDefault.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): FeatureSetDefaults_FeatureSetEditionDefault { const message = createBaseFeatureSetDefaults_FeatureSetEditionDefault(); message.edition = object.edition ?? 0; - message.overridableFeatures = - object.overridableFeatures !== undefined && object.overridableFeatures !== null - ? FeatureSet.fromPartial(object.overridableFeatures) - : undefined; - message.fixedFeatures = - object.fixedFeatures !== undefined && object.fixedFeatures !== null - ? FeatureSet.fromPartial(object.fixedFeatures) - : undefined; + message.overridableFeatures = (object.overridableFeatures !== undefined && object.overridableFeatures !== null) + ? FeatureSet.fromPartial(object.overridableFeatures) + : undefined; + message.fixedFeatures = (object.fixedFeatures !== undefined && object.fixedFeatures !== null) + ? FeatureSet.fromPartial(object.fixedFeatures) + : undefined; return message; }, }; @@ -5922,7 +6077,7 @@ export const SourceCodeInfo: MessageFns = { toJSON(message: SourceCodeInfo): unknown { const obj: any = {}; if (message.location?.length) { - obj.location = message.location.map(e => SourceCodeInfo_Location.toJSON(e)); + obj.location = message.location.map((e) => SourceCodeInfo_Location.toJSON(e)); } return obj; }, @@ -5932,7 +6087,7 @@ export const SourceCodeInfo: MessageFns = { }, fromPartial, I>>(object: I): SourceCodeInfo { const message = createBaseSourceCodeInfo(); - message.location = object.location?.map(e => SourceCodeInfo_Location.fromPartial(e)) || []; + message.location = object.location?.map((e) => SourceCodeInfo_Location.fromPartial(e)) || []; return message; }, }; @@ -6051,10 +6206,10 @@ export const SourceCodeInfo_Location: MessageFns = { toJSON(message: SourceCodeInfo_Location): unknown { const obj: any = {}; if (message.path?.length) { - obj.path = message.path.map(e => Math.round(e)); + obj.path = message.path.map((e) => Math.round(e)); } if (message.span?.length) { - obj.span = message.span.map(e => Math.round(e)); + obj.span = message.span.map((e) => Math.round(e)); } if (message.leadingComments !== undefined && message.leadingComments !== "") { obj.leadingComments = message.leadingComments; @@ -6073,11 +6228,11 @@ export const SourceCodeInfo_Location: MessageFns = { }, fromPartial, I>>(object: I): SourceCodeInfo_Location { const message = createBaseSourceCodeInfo_Location(); - message.path = object.path?.map(e => e) || []; - message.span = object.span?.map(e => e) || []; + message.path = object.path?.map((e) => e) || []; + message.span = object.span?.map((e) => e) || []; message.leadingComments = object.leadingComments ?? ""; message.trailingComments = object.trailingComments ?? ""; - message.leadingDetachedComments = object.leadingDetachedComments?.map(e => e) || []; + message.leadingDetachedComments = object.leadingDetachedComments?.map((e) => e) || []; return message; }, }; @@ -6128,7 +6283,7 @@ export const GeneratedCodeInfo: MessageFns = { toJSON(message: GeneratedCodeInfo): unknown { const obj: any = {}; if (message.annotation?.length) { - obj.annotation = message.annotation.map(e => GeneratedCodeInfo_Annotation.toJSON(e)); + obj.annotation = message.annotation.map((e) => GeneratedCodeInfo_Annotation.toJSON(e)); } return obj; }, @@ -6138,7 +6293,7 @@ export const GeneratedCodeInfo: MessageFns = { }, fromPartial, I>>(object: I): GeneratedCodeInfo { const message = createBaseGeneratedCodeInfo(); - message.annotation = object.annotation?.map(e => GeneratedCodeInfo_Annotation.fromPartial(e)) || []; + message.annotation = object.annotation?.map((e) => GeneratedCodeInfo_Annotation.fromPartial(e)) || []; return message; }, }; @@ -6243,7 +6398,7 @@ export const GeneratedCodeInfo_Annotation: MessageFns Math.round(e)); + obj.path = message.path.map((e) => Math.round(e)); } if (message.sourceFile !== undefined && message.sourceFile !== "") { obj.sourceFile = message.sourceFile; @@ -6265,7 +6420,7 @@ export const GeneratedCodeInfo_Annotation: MessageFns, I>>(object: I): GeneratedCodeInfo_Annotation { const message = createBaseGeneratedCodeInfo_Annotation(); - message.path = object.path?.map(e => e) || []; + message.path = object.path?.map((e) => e) || []; message.sourceFile = object.sourceFile ?? ""; message.begin = object.begin ?? 0; message.end = object.end ?? 0; @@ -6292,7 +6447,7 @@ function base64FromBytes(arr: Uint8Array): string { return globalThis.Buffer.from(arr).toString("base64"); } else { const bin: string[] = []; - arr.forEach(byte => { + arr.forEach((byte) => { bin.push(globalThis.String.fromCharCode(byte)); }); return globalThis.btoa(bin.join("")); @@ -6301,19 +6456,14 @@ function base64FromBytes(arr: Uint8Array): string { type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; -export type DeepPartial = T extends Builtin - ? T - : T extends globalThis.Array - ? globalThis.Array> - : T extends ReadonlyArray - ? ReadonlyArray> - : T extends {} - ? { [K in keyof T]?: DeepPartial } - : Partial; +export type DeepPartial = T extends Builtin ? T + : T extends globalThis.Array ? globalThis.Array> + : T extends ReadonlyArray ? ReadonlyArray> + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; type KeysOfUnion = T extends T ? keyof T : never; -export type Exact = P extends Builtin - ? P +export type Exact = P extends Builtin ? P : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; function longToNumber(int64: { toString(): string }): number { diff --git a/core/gui/src/app/common/type/proto/scalapb/scalapb.ts b/core/gui/src/app/common/type/proto/scalapb/scalapb.ts index f1432ae38bb..ef9762b0ca8 100644 --- a/core/gui/src/app/common/type/proto/scalapb/scalapb.ts +++ b/core/gui/src/app/common/type/proto/scalapb/scalapb.ts @@ -20,7 +20,7 @@ // Code generated by protoc-gen-ts_proto. DO NOT EDIT. // versions: // protoc-gen-ts_proto v2.2.0 -// protoc v5.28.3 +// protoc v5.29.3 // source: scalapb/scalapb.proto /* eslint-disable */ @@ -70,14 +70,18 @@ export function matchTypeToJSON(object: MatchType): string { export interface ScalaPbOptions { /** If set then it overrides the java_package and package. */ - packageName?: string | undefined; + packageName?: + | string + | undefined; /** * If true, the compiler does not append the proto base file name * into the generated package name. If false (the default), the * generated scala package name is the package_name.basename where * basename is the proto file name without the .proto extension. */ - flatPackage?: boolean | undefined; + flatPackage?: + | boolean + | undefined; /** * Adds the following imports at the top of the file (this is meant * to provide implicit TypeMappers) @@ -92,65 +96,93 @@ export interface ScalaPbOptions { * If true, all messages and enums (but not services) will be written * to a single Scala file. */ - singleFile?: boolean | undefined; + singleFile?: + | boolean + | undefined; /** * By default, wrappers defined at * https://github.com/google/protobuf/blob/master/src/google/protobuf/wrappers.proto, * are mapped to an Option[T] where T is a primitive type. When this field * is set to true, we do not perform this transformation. */ - noPrimitiveWrappers?: boolean | undefined; + noPrimitiveWrappers?: + | boolean + | undefined; /** * DEPRECATED. In ScalaPB <= 0.5.47, it was necessary to explicitly enable * primitive_wrappers. This field remains here for backwards compatibility, * but it has no effect on generated code. It is an error to set both * `primitive_wrappers` and `no_primitive_wrappers`. */ - primitiveWrappers?: boolean | undefined; + primitiveWrappers?: + | boolean + | undefined; /** * Scala type to be used for repeated fields. If unspecified, * `scala.collection.Seq` will be used. */ - collectionType?: string | undefined; + collectionType?: + | string + | undefined; /** * If set to true, all generated messages in this file will preserve unknown * fields. */ - preserveUnknownFields?: boolean | undefined; + preserveUnknownFields?: + | boolean + | undefined; /** * If defined, sets the name of the file-level object that would be generated. This * object extends `GeneratedFileObject` and contains descriptors, and list of message * and enum companions. */ - objectName?: string | undefined; + objectName?: + | string + | undefined; /** Experimental: scope to apply the given options. */ - scope?: ScalaPbOptions_OptionsScope | undefined; + scope?: + | ScalaPbOptions_OptionsScope + | undefined; /** If true, lenses will be generated. */ - lenses?: boolean | undefined; + lenses?: + | boolean + | undefined; /** * If true, then source-code info information will be included in the * generated code - normally the source code info is cleared out to reduce * code size. The source code info is useful for extracting source code * location from the descriptors as well as comments. */ - retainSourceCodeInfo?: boolean | undefined; + retainSourceCodeInfo?: + | boolean + | undefined; /** * Scala type to be used for maps. If unspecified, * `scala.collection.immutable.Map` will be used. */ - mapType?: string | undefined; + mapType?: + | string + | undefined; /** If true, no default values will be generated in message constructors. */ noDefaultValuesInConstructor?: boolean | undefined; - enumValueNaming?: ScalaPbOptions_EnumValueNaming | undefined; + enumValueNaming?: + | ScalaPbOptions_EnumValueNaming + | undefined; /** * Indicate if prefix (enum name + optional underscore) should be removed in scala code * Strip is applied before enum value naming changes. */ - enumStripPrefix?: boolean | undefined; + enumStripPrefix?: + | boolean + | undefined; /** Scala type to use for bytes fields. */ - bytesType?: string | undefined; + bytesType?: + | string + | undefined; /** Enable java conversions for this file. */ - javaConversions?: boolean | undefined; + javaConversions?: + | boolean + | undefined; /** List of message options to apply to some messages. */ auxMessageOptions: ScalaPbOptions_AuxMessageOptions[]; /** List of message options to apply to some fields. */ @@ -166,9 +198,13 @@ export interface ScalaPbOptions { * Ignores all transformations for this file. This is meant to allow specific files to * opt out from transformations inherited through package-scoped options. */ - ignoreAllTransformations?: boolean | undefined; + ignoreAllTransformations?: + | boolean + | undefined; /** If true, getters will be generated. */ - getters?: boolean | undefined; + getters?: + | boolean + | undefined; /** * For use in tests only. Inhibit Java conversions even when when generator parameters * request for it. @@ -255,7 +291,9 @@ export function scalaPbOptions_EnumValueNamingToJSON(object: ScalaPbOptions_Enum */ export interface ScalaPbOptions_AuxMessageOptions { /** The fully-qualified name of the message in the proto name space. */ - target?: string | undefined; + target?: + | string + | undefined; /** * Options to apply to the message. If there are any options defined on the target message * they take precedence over the options. @@ -270,7 +308,9 @@ export interface ScalaPbOptions_AuxMessageOptions { */ export interface ScalaPbOptions_AuxFieldOptions { /** The fully-qualified name of the field in the proto name space. */ - target?: string | undefined; + target?: + | string + | undefined; /** * Options to apply to the field. If there are any options defined on the target message * they take precedence over the options. @@ -285,7 +325,9 @@ export interface ScalaPbOptions_AuxFieldOptions { */ export interface ScalaPbOptions_AuxEnumOptions { /** The fully-qualified name of the enum in the proto name space. */ - target?: string | undefined; + target?: + | string + | undefined; /** * Options to apply to the enum. If there are any options defined on the target enum * they take precedence over the options. @@ -300,7 +342,9 @@ export interface ScalaPbOptions_AuxEnumOptions { */ export interface ScalaPbOptions_AuxEnumValueOptions { /** The fully-qualified name of the enum value in the proto name space. */ - target?: string | undefined; + target?: + | string + | undefined; /** * Options to apply to the enum value. If there are any options defined on * the target enum value they take precedence over the options. @@ -319,7 +363,9 @@ export interface MessageOptions { * All instances of this message will be converted to this type. An implicit TypeMapper * must be present. */ - type?: string | undefined; + type?: + | string + | undefined; /** Custom annotations to add to the companion object of the generated class. */ companionAnnotations: string[]; /** Additional classes and traits to mix in to generated sealed_oneof base trait. */ @@ -328,7 +374,9 @@ export interface MessageOptions { * If true, when this message is used as an optional field, do not wrap it in an `Option`. * This is equivalent of setting `(field).no_box` to true on each field with the message type. */ - noBox?: boolean | undefined; + noBox?: + | boolean + | undefined; /** Custom annotations to add to the generated `unknownFields` case class field. */ unknownFieldsAnnotations: string[]; } @@ -339,13 +387,17 @@ export interface MessageOptions { */ export interface Collection { /** Type of the collection */ - type?: string | undefined; + type?: + | string + | undefined; /** * Set to true if this collection type is not allowed to be empty, for example * cats.data.NonEmptyList. When true, ScalaPB will not generate `clearX` for the repeated * field and not provide a default argument in the constructor. */ - nonEmpty?: boolean | undefined; + nonEmpty?: + | boolean + | undefined; /** * An Adapter is a Scala object available at runtime that provides certain static methods * that can operate on this collection type. @@ -355,20 +407,26 @@ export interface Collection { export interface FieldOptions { type?: string | undefined; - scalaName?: string | undefined; + scalaName?: + | string + | undefined; /** * Can be specified only if this field is repeated. If unspecified, * it falls back to the file option named `collection_type`, which defaults * to `scala.collection.Seq`. */ collectionType?: string | undefined; - collection?: Collection | undefined; + collection?: + | Collection + | undefined; /** * If the field is a map, you can specify custom Scala types for the key * or value. */ keyType?: string | undefined; - valueType?: string | undefined; + valueType?: + | string + | undefined; /** Custom annotations to add to the field. */ annotations: string[]; /** @@ -376,9 +434,13 @@ export interface FieldOptions { * it falls back to the file option named `map_type` which defaults to * `scala.collection.immutable.Map` */ - mapType?: string | undefined; + mapType?: + | string + | undefined; /** Do not box this value in Option[T]. If set, this overrides MessageOptions.no_box */ - noBox?: boolean | undefined; + noBox?: + | boolean + | undefined; /** * Like no_box it does not box a value in Option[T], but also fails parsing when a value * is not provided. This enables to emulate required fields in proto3. @@ -395,7 +457,9 @@ export interface EnumOptions { * All instances of this enum will be converted to this type. An implicit TypeMapper * must be present. */ - type?: string | undefined; + type?: + | string + | undefined; /** Custom annotations to add to the generated enum's base class. */ baseAnnotations: string[]; /** Custom annotations to add to the generated trait. */ @@ -408,7 +472,9 @@ export interface EnumValueOptions { /** Additional classes and traits to mix in to an individual enum value. */ extends: string[]; /** Name in Scala to use for this enum value. */ - scalaName?: string | undefined; + scalaName?: + | string + | undefined; /** Custom annotations to add to the generated case object for this enum value. */ annotations: string[]; } @@ -886,22 +952,22 @@ export const ScalaPbOptions: MessageFns = { obj.javaConversions = message.javaConversions; } if (message.auxMessageOptions?.length) { - obj.auxMessageOptions = message.auxMessageOptions.map(e => ScalaPbOptions_AuxMessageOptions.toJSON(e)); + obj.auxMessageOptions = message.auxMessageOptions.map((e) => ScalaPbOptions_AuxMessageOptions.toJSON(e)); } if (message.auxFieldOptions?.length) { - obj.auxFieldOptions = message.auxFieldOptions.map(e => ScalaPbOptions_AuxFieldOptions.toJSON(e)); + obj.auxFieldOptions = message.auxFieldOptions.map((e) => ScalaPbOptions_AuxFieldOptions.toJSON(e)); } if (message.auxEnumOptions?.length) { - obj.auxEnumOptions = message.auxEnumOptions.map(e => ScalaPbOptions_AuxEnumOptions.toJSON(e)); + obj.auxEnumOptions = message.auxEnumOptions.map((e) => ScalaPbOptions_AuxEnumOptions.toJSON(e)); } if (message.auxEnumValueOptions?.length) { - obj.auxEnumValueOptions = message.auxEnumValueOptions.map(e => ScalaPbOptions_AuxEnumValueOptions.toJSON(e)); + obj.auxEnumValueOptions = message.auxEnumValueOptions.map((e) => ScalaPbOptions_AuxEnumValueOptions.toJSON(e)); } if (message.preprocessors?.length) { obj.preprocessors = message.preprocessors; } if (message.fieldTransformations?.length) { - obj.fieldTransformations = message.fieldTransformations.map(e => FieldTransformation.toJSON(e)); + obj.fieldTransformations = message.fieldTransformations.map((e) => FieldTransformation.toJSON(e)); } if (message.ignoreAllTransformations !== undefined && message.ignoreAllTransformations !== false) { obj.ignoreAllTransformations = message.ignoreAllTransformations; @@ -922,8 +988,8 @@ export const ScalaPbOptions: MessageFns = { const message = createBaseScalaPbOptions(); message.packageName = object.packageName ?? ""; message.flatPackage = object.flatPackage ?? false; - message.import = object.import?.map(e => e) || []; - message.preamble = object.preamble?.map(e => e) || []; + message.import = object.import?.map((e) => e) || []; + message.preamble = object.preamble?.map((e) => e) || []; message.singleFile = object.singleFile ?? false; message.noPrimitiveWrappers = object.noPrimitiveWrappers ?? false; message.primitiveWrappers = object.primitiveWrappers ?? false; @@ -939,14 +1005,14 @@ export const ScalaPbOptions: MessageFns = { message.enumStripPrefix = object.enumStripPrefix ?? false; message.bytesType = object.bytesType ?? ""; message.javaConversions = object.javaConversions ?? false; - message.auxMessageOptions = - object.auxMessageOptions?.map(e => ScalaPbOptions_AuxMessageOptions.fromPartial(e)) || []; - message.auxFieldOptions = object.auxFieldOptions?.map(e => ScalaPbOptions_AuxFieldOptions.fromPartial(e)) || []; - message.auxEnumOptions = object.auxEnumOptions?.map(e => ScalaPbOptions_AuxEnumOptions.fromPartial(e)) || []; + message.auxMessageOptions = object.auxMessageOptions?.map((e) => ScalaPbOptions_AuxMessageOptions.fromPartial(e)) || + []; + message.auxFieldOptions = object.auxFieldOptions?.map((e) => ScalaPbOptions_AuxFieldOptions.fromPartial(e)) || []; + message.auxEnumOptions = object.auxEnumOptions?.map((e) => ScalaPbOptions_AuxEnumOptions.fromPartial(e)) || []; message.auxEnumValueOptions = - object.auxEnumValueOptions?.map(e => ScalaPbOptions_AuxEnumValueOptions.fromPartial(e)) || []; - message.preprocessors = object.preprocessors?.map(e => e) || []; - message.fieldTransformations = object.fieldTransformations?.map(e => FieldTransformation.fromPartial(e)) || []; + object.auxEnumValueOptions?.map((e) => ScalaPbOptions_AuxEnumValueOptions.fromPartial(e)) || []; + message.preprocessors = object.preprocessors?.map((e) => e) || []; + message.fieldTransformations = object.fieldTransformations?.map((e) => FieldTransformation.fromPartial(e)) || []; message.ignoreAllTransformations = object.ignoreAllTransformations ?? false; message.getters = object.getters ?? true; message.testOnlyNoJavaConversions = object.testOnlyNoJavaConversions ?? false; @@ -1018,17 +1084,18 @@ export const ScalaPbOptions_AuxMessageOptions: MessageFns, I>>( - base?: I + base?: I, ): ScalaPbOptions_AuxMessageOptions { return ScalaPbOptions_AuxMessageOptions.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): ScalaPbOptions_AuxMessageOptions { const message = createBaseScalaPbOptions_AuxMessageOptions(); message.target = object.target ?? ""; - message.options = - object.options !== undefined && object.options !== null ? MessageOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? MessageOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -1100,12 +1167,13 @@ export const ScalaPbOptions_AuxFieldOptions: MessageFns, I>>( - object: I + object: I, ): ScalaPbOptions_AuxFieldOptions { const message = createBaseScalaPbOptions_AuxFieldOptions(); message.target = object.target ?? ""; - message.options = - object.options !== undefined && object.options !== null ? FieldOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? FieldOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -1177,12 +1245,13 @@ export const ScalaPbOptions_AuxEnumOptions: MessageFns, I>>( - object: I + object: I, ): ScalaPbOptions_AuxEnumOptions { const message = createBaseScalaPbOptions_AuxEnumOptions(); message.target = object.target ?? ""; - message.options = - object.options !== undefined && object.options !== null ? EnumOptions.fromPartial(object.options) : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? EnumOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -1251,19 +1320,18 @@ export const ScalaPbOptions_AuxEnumValueOptions: MessageFns, I>>( - base?: I + base?: I, ): ScalaPbOptions_AuxEnumValueOptions { return ScalaPbOptions_AuxEnumValueOptions.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): ScalaPbOptions_AuxEnumValueOptions { const message = createBaseScalaPbOptions_AuxEnumValueOptions(); message.target = object.target ?? ""; - message.options = - object.options !== undefined && object.options !== null - ? EnumValueOptions.fromPartial(object.options) - : undefined; + message.options = (object.options !== undefined && object.options !== null) + ? EnumValueOptions.fromPartial(object.options) + : undefined; return message; }, }; @@ -1439,14 +1507,14 @@ export const MessageOptions: MessageFns = { }, fromPartial, I>>(object: I): MessageOptions { const message = createBaseMessageOptions(); - message.extends = object.extends?.map(e => e) || []; - message.companionExtends = object.companionExtends?.map(e => e) || []; - message.annotations = object.annotations?.map(e => e) || []; + message.extends = object.extends?.map((e) => e) || []; + message.companionExtends = object.companionExtends?.map((e) => e) || []; + message.annotations = object.annotations?.map((e) => e) || []; message.type = object.type ?? ""; - message.companionAnnotations = object.companionAnnotations?.map(e => e) || []; - message.sealedOneofExtends = object.sealedOneofExtends?.map(e => e) || []; + message.companionAnnotations = object.companionAnnotations?.map((e) => e) || []; + message.sealedOneofExtends = object.sealedOneofExtends?.map((e) => e) || []; message.noBox = object.noBox ?? false; - message.unknownFieldsAnnotations = object.unknownFieldsAnnotations?.map(e => e) || []; + message.unknownFieldsAnnotations = object.unknownFieldsAnnotations?.map((e) => e) || []; return message; }, }; @@ -1736,13 +1804,12 @@ export const FieldOptions: MessageFns = { message.type = object.type ?? ""; message.scalaName = object.scalaName ?? ""; message.collectionType = object.collectionType ?? ""; - message.collection = - object.collection !== undefined && object.collection !== null - ? Collection.fromPartial(object.collection) - : undefined; + message.collection = (object.collection !== undefined && object.collection !== null) + ? Collection.fromPartial(object.collection) + : undefined; message.keyType = object.keyType ?? ""; message.valueType = object.valueType ?? ""; - message.annotations = object.annotations?.map(e => e) || []; + message.annotations = object.annotations?.map((e) => e) || []; message.mapType = object.mapType ?? ""; message.noBox = object.noBox ?? false; message.required = object.required ?? false; @@ -1889,12 +1956,12 @@ export const EnumOptions: MessageFns = { }, fromPartial, I>>(object: I): EnumOptions { const message = createBaseEnumOptions(); - message.extends = object.extends?.map(e => e) || []; - message.companionExtends = object.companionExtends?.map(e => e) || []; + message.extends = object.extends?.map((e) => e) || []; + message.companionExtends = object.companionExtends?.map((e) => e) || []; message.type = object.type ?? ""; - message.baseAnnotations = object.baseAnnotations?.map(e => e) || []; - message.recognizedAnnotations = object.recognizedAnnotations?.map(e => e) || []; - message.unrecognizedAnnotations = object.unrecognizedAnnotations?.map(e => e) || []; + message.baseAnnotations = object.baseAnnotations?.map((e) => e) || []; + message.recognizedAnnotations = object.recognizedAnnotations?.map((e) => e) || []; + message.unrecognizedAnnotations = object.unrecognizedAnnotations?.map((e) => e) || []; return message; }, }; @@ -1983,9 +2050,9 @@ export const EnumValueOptions: MessageFns = { }, fromPartial, I>>(object: I): EnumValueOptions { const message = createBaseEnumValueOptions(); - message.extends = object.extends?.map(e => e) || []; + message.extends = object.extends?.map((e) => e) || []; message.scalaName = object.scalaName ?? ""; - message.annotations = object.annotations?.map(e => e) || []; + message.annotations = object.annotations?.map((e) => e) || []; return message; }, }; @@ -2058,7 +2125,7 @@ export const OneofOptions: MessageFns = { }, fromPartial, I>>(object: I): OneofOptions { const message = createBaseOneofOptions(); - message.extends = object.extends?.map(e => e) || []; + message.extends = object.extends?.map((e) => e) || []; message.scalaName = object.scalaName ?? ""; return message; }, @@ -2146,10 +2213,11 @@ export const FieldTransformation: MessageFns = { }, fromPartial, I>>(object: I): FieldTransformation { const message = createBaseFieldTransformation(); - message.when = - object.when !== undefined && object.when !== null ? FieldDescriptorProto.fromPartial(object.when) : undefined; + message.when = (object.when !== undefined && object.when !== null) + ? FieldDescriptorProto.fromPartial(object.when) + : undefined; message.matchType = object.matchType ?? 0; - message.set = object.set !== undefined && object.set !== null ? FieldOptions1.fromPartial(object.set) : undefined; + message.set = (object.set !== undefined && object.set !== null) ? FieldOptions1.fromPartial(object.set) : undefined; return message; }, }; @@ -2196,9 +2264,9 @@ export const PreprocessorOutput: MessageFns = { return { optionsByFile: isObject(object.optionsByFile) ? Object.entries(object.optionsByFile).reduce<{ [key: string]: ScalaPbOptions }>((acc, [key, value]) => { - acc[key] = ScalaPbOptions.fromJSON(value); - return acc; - }, {}) + acc[key] = ScalaPbOptions.fromJSON(value); + return acc; + }, {}) : {}, }; }, @@ -2229,7 +2297,7 @@ export const PreprocessorOutput: MessageFns = { } return acc; }, - {} + {}, ); return message; }, @@ -2299,36 +2367,32 @@ export const PreprocessorOutput_OptionsByFileEntry: MessageFns, I>>( - base?: I + base?: I, ): PreprocessorOutput_OptionsByFileEntry { return PreprocessorOutput_OptionsByFileEntry.fromPartial(base ?? ({} as any)); }, fromPartial, I>>( - object: I + object: I, ): PreprocessorOutput_OptionsByFileEntry { const message = createBasePreprocessorOutput_OptionsByFileEntry(); message.key = object.key ?? ""; - message.value = - object.value !== undefined && object.value !== null ? ScalaPbOptions.fromPartial(object.value) : undefined; + message.value = (object.value !== undefined && object.value !== null) + ? ScalaPbOptions.fromPartial(object.value) + : undefined; return message; }, }; type Builtin = Date | Function | Uint8Array | string | number | boolean | undefined; -export type DeepPartial = T extends Builtin - ? T - : T extends globalThis.Array - ? globalThis.Array> - : T extends ReadonlyArray - ? ReadonlyArray> - : T extends {} - ? { [K in keyof T]?: DeepPartial } - : Partial; +export type DeepPartial = T extends Builtin ? T + : T extends globalThis.Array ? globalThis.Array> + : T extends ReadonlyArray ? ReadonlyArray> + : T extends {} ? { [K in keyof T]?: DeepPartial } + : Partial; type KeysOfUnion = T extends T ? keyof T : never; -export type Exact = P extends Builtin - ? P +export type Exact = P extends Builtin ? P : P & { [K in keyof P]: Exact } & { [K in Exclude>]: never }; function isObject(value: any): boolean { diff --git a/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.html b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.html new file mode 100644 index 00000000000..3c656606f6b --- /dev/null +++ b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.html @@ -0,0 +1,170 @@ +
+
+ + {{ columnProfile.columnName }} + {{ columnProfile.dataType }} + {{ columnProfile.dataLabel || "N/A" }} + {{ columnProfile.categorical ? "Yes" : "No" }} + {{ columnProfile.order || "N/A" }} + + +

+ Statistics +

+ + + + Metric + Value + + + + + {{ stat.metric }} + {{ stat.value }} + + + + +
+

Top 10 Categorical Distribution

+ + +
+ +
+
+

Data Cleaning Suggestions

+ +
+
+ + + Powered by Data Copilot + +
+
+ +
+ + Loading data cleaning suggestions... +
+ +
+ No data cleaning suggestions available for this column. +
+ +
    +
  • +
    + {{ suggestion.suggestion }} + +
    +
    + +
    + + +
    +
    +
  • +
+
+
+ + +
+ +

Select a column header in the result table to view its profile.

+
+
diff --git a/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.scss b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.scss new file mode 100644 index 00000000000..06636306885 --- /dev/null +++ b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.scss @@ -0,0 +1,216 @@ +:host { + display: block; + padding: 10px; + overflow-y: auto; + height: 100%; // Ensure it takes available height in left panel + width: 100%; // Ensure host takes up available width from left-panel +} + +.column-profile-container { + h4.section-header { + margin-top: 16px; + margin-bottom: 8px; + font-size: 1em; + font-weight: 500; + } + + .chart-section { + margin-top: 16px; + width: 100%; // Make chart section take full available width + box-sizing: border-box; // Include padding and border in the element's total width and height + + h4 { + // If the h4 for "Top 10..." is inside .chart-section + margin-bottom: 8px; + } + + ngx-charts-bar-vertical { + display: block; + width: 100% !important; // Important to override potential inline styles from ngx-charts + height: auto; // Or a fixed height like 250px, 300px + box-sizing: border-box; + } + } + + .data-cleaning-suggestions-section { + margin-top: 16px; + + .suggestions-header-main-line { + display: flex; + justify-content: space-between; + align-items: center; + width: 100%; + margin-bottom: 2px; + } + + .copilot-attribution-wrapper { + margin-bottom: 8px; + } + + .copilot-attribution { + display: flex; + align-items: center; + font-size: 0.75em; + color: #777; + font-family: "Segoe UI", Tahoma, Geneva, Verdana, sans-serif; + font-style: italic; + + i[nz-icon] { + margin-right: 5px; + font-size: 1.2em; + color: #007bff; + } + } + + h4 { + margin-bottom: 0; + } + + .no-suggestions-message { + padding: 10px; + background-color: #f0f0f0; + border: 1px solid #d9d9d9; + border-radius: 4px; + text-align: center; + color: #555; + font-style: italic; + } + } + + // ========================= + // Suggestion List & Items + // ========================= + .suggestions-list { + padding: 0; + list-style-type: none; + margin: 0; + } + + // Style each suggestion item regardless of parent container + .suggestion-item { + margin-bottom: 6px; + color: #333; + border-radius: 3px; + cursor: pointer; + display: flex; + flex-direction: column; + // Apply transitions for hover/active effects + transition: background-color 0.25s ease, border-color 0.25s ease, transform 0.25s ease, box-shadow 0.25s ease; + + // Default appearance + background-color: #fffbe6; // light yellow + border: 1px solid #ffe58f; + padding: 10px 14px; + + &:last-child { + margin-bottom: 0; + } + + // Hover (not expanded) + &:not(.expanded):hover { + background-color: #ffecb3; + border-color: #ffd666; + transform: translateY(-2px); + box-shadow: 0 2px 6px rgba(0, 0, 0, 0.15); + color: #000; + } + + // Expanded state + &.expanded { + background-color: #fff9db; + border-color: #ffe58f; + padding-bottom: 12px; + box-shadow: 0 2px 4px rgba(0, 0, 0, 0.1); + + &:hover { + background-color: #ffefc2; + } + + .suggestion-details-wrapper { + padding-top: 10px; + margin-top: 10px; + border-top: 1px dashed #dcdcdc; + } + + .suggestion-details-area { + width: 100%; + box-sizing: border-box; + min-height: 160px; + font-family: monospace; + font-size: 0.85em; + padding: 8px; + border: 1px solid #ccc; + border-radius: 3px; + resize: vertical; + background-color: #fdfdfd; + color: #333; + margin-bottom: 10px; + } + + .suggestion-actions { + display: flex; + justify-content: flex-end; + gap: 8px; + } + } + + // Inner elements common to both collapsed & expanded states + .suggestion-summary { + display: flex; + justify-content: space-between; + align-items: center; + width: 100%; + } + + .expand-icon { + margin-left: 8px; + font-size: 0.8em; + color: #555; + } + } +} + +.no-column-placeholder { + display: flex; + flex-direction: column; + align-items: center; + justify-content: center; + height: 100%; + text-align: center; + color: #888; + font-size: 0.9em; + + i { + font-size: 24px; + margin-bottom: 10px; + } +} + +// Remove or comment out the ngx-charts deep overrides for font sizes for now, +// unless they are still specifically needed after width adjustments. +// :host ::ng-deep .ngx-charts { +// text { +// font-size: 10px !important; +// } +// .x.axis .tick text, .y.axis .tick text { +// font-size: 9px !important; +// } +// } + +.loading-suggestions-message { + padding: 10px; + text-align: center; + background-color: #fffbe6; // light yellow background + border: 1px solid #ffe58f; // yellow border to match suggestion items + color: #ad8b00; // dark yellow/brown text for contrast + border-radius: 4px; + display: flex; + align-items: center; + justify-content: center; + gap: 6px; + font-weight: 500; + + .loading-icon { + font-size: 16px; + } +} diff --git a/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.ts b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.ts new file mode 100644 index 00000000000..4f10d636e6a --- /dev/null +++ b/core/gui/src/app/workspace/component/left-panel/column-profile-frame/column-profile-frame.component.ts @@ -0,0 +1,352 @@ +import { Component, OnInit, OnDestroy, ChangeDetectorRef, TemplateRef, ViewChild, ElementRef } from "@angular/core"; +import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; +import { NzModalService } from "ng-zorro-antd/modal"; +import { + ColumnProfile, + TableProfile, +} from "../../../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; +import { WorkflowSuggestionService } from "../../../service/workflow-suggestion/workflow-suggestion.service"; +import { ColumnProfileService, SelectedColumnInfo } from "../../../service/column-profile/column-profile.service"; +import { finalize } from "rxjs/operators"; +import { isDefined } from "../../../../common/util/predicate"; +import { + WorkflowDataCleaningSuggestion, + WorkflowDataCleaningSuggestionList, + WorkflowSuggestion, + WorkflowSuggestionList, +} from "src/app/workspace/types/workflow-suggestion.interface"; +import { SchemaAttribute } from "src/app/workspace/types/workflow-compiling.interface"; +import { WorkflowActionService } from "../../../service/workflow-graph/model/workflow-action.service"; +import { WorkflowCompilingService } from "../../../service/compile-workflow/workflow-compiling.service"; +import { SuggestionActionService } from "../../../service/suggestion-action/suggestion-action.service"; +import { ExecuteWorkflowService } from "../../../service/execute-workflow/execute-workflow.service"; +import { NzMessageService } from "ng-zorro-antd/message"; + +interface DisplayableDataCleaningSuggestion extends WorkflowDataCleaningSuggestion { + isExpanded?: boolean; + details: string; // To store a string representation of changes or other details + isAccepting?: boolean; // To show loading on the accept button +} + +@UntilDestroy() +@Component({ + selector: "texera-column-profile-frame", + templateUrl: "./column-profile-frame.component.html", + styleUrls: ["./column-profile-frame.component.scss"], +}) +export class ColumnProfileFrameComponent implements OnInit { + public selectedColumnInfo: SelectedColumnInfo | null = null; + public columnProfile: ColumnProfile | undefined; + public tableProfile: TableProfile | undefined; + public tableSchema: ReadonlyArray | undefined; + + public columnNumericStatsForTable: Array<{ metric: string; value: string | number | undefined }> = []; + public barChartData: Array<{ name: string; value: number }> = []; + + // ngx-charts options + public view: [number, number] = [200, 250]; // Initial value, will be overridden if panel width is available + public showXAxis = true; + public showYAxis = true; + public gradient = false; + public showLegend = false; + public showXAxisLabel = true; + public xAxisLabel = "Category"; + public showYAxisLabel = true; + public yAxisLabel = "Count"; + + // For Data Cleaning Suggestions + public dataCleaningSuggestions: DisplayableDataCleaningSuggestion[] = []; + public isLoadingDataCleaningSuggestions: boolean = false; + private dataCleaningSuggestionsCache: Map = new Map(); + public lastFetchedSuggestionsForColumn: string | null = null; + + constructor( + private columnProfileService: ColumnProfileService, + private workflowSuggestionService: WorkflowSuggestionService, + private workflowActionService: WorkflowActionService, + private workflowCompilingService: WorkflowCompilingService, + private executeWorkflowService: ExecuteWorkflowService, + private suggestionActionService: SuggestionActionService, + private messageService: NzMessageService, + private changeDetectorRef: ChangeDetectorRef, + private elRef: ElementRef // Inject ElementRef to get host width + ) {} + + ngOnInit(): void { + this.columnProfileService + .getSelectedColumnStream() + .pipe(untilDestroyed(this)) + .subscribe(selectedInfo => { + this.selectedColumnInfo = selectedInfo; + if (selectedInfo) { + this.columnProfile = selectedInfo.columnProfile; + this.tableProfile = selectedInfo.tableProfile; + this.tableSchema = selectedInfo.schema; + this.loadDisplayData(); + } else { + this.columnProfile = undefined; + this.tableProfile = undefined; + this.tableSchema = undefined; + this.resetDisplayData(); + } + this.changeDetectorRef.detectChanges(); + }); + + this.updateChartViewWidth(); + } + + // Add AfterViewChecked to update chart width if panel resizes + // This is a simple way; a more robust solution might use ResizeObserver + ngAfterViewChecked(): void { + this.updateChartViewWidth(); + } + + private updateChartViewWidth(): void { + // Attempt to set chart width based on parent container + // This requires the .chart-section or its parent to have a defined width + const hostElement = this.elRef.nativeElement; + const chartContainer = hostElement.querySelector(".chart-section"); + if (chartContainer) { + const containerWidth = chartContainer.clientWidth; + if (containerWidth > 0) { + // Subtract some padding/margin if chart has internal margins + const chartWidth = Math.max(150, containerWidth - 20); // Ensure a minimum width + if (this.view[0] !== chartWidth) { + this.view = [chartWidth, this.view[1]]; // Keep existing height + this.changeDetectorRef.detectChanges(); // Trigger change detection if view updated + } + } + } else if (hostElement.clientWidth > 0 && this.view[0] !== hostElement.clientWidth - 20) { + // Fallback to host element width if .chart-section not found or has no width yet + const chartWidth = Math.max(150, hostElement.clientWidth - 40); // Wider padding for host + if (this.view[0] !== chartWidth) { + this.view = [chartWidth, this.view[1]]; + this.changeDetectorRef.detectChanges(); + } + } + } + + private loadDisplayData(): void { + if (!this.columnProfile) { + this.resetDisplayData(); + return; + } + this.prepareColumnNumericStats(this.columnProfile); + if (this.columnProfile.categorical && this.columnProfile.statistics) { + this.prepareBarChartData(this.columnProfile.statistics.categoricalCount); + } else { + this.barChartData = []; + } + // this.fetchOrGetCachedDataCleaningSuggestions(this.columnProfile); + } + + private resetDisplayData(): void { + this.columnNumericStatsForTable = []; + this.barChartData = []; + this.dataCleaningSuggestions = []; + this.isLoadingDataCleaningSuggestions = false; + this.lastFetchedSuggestionsForColumn = null; + this.tableSchema = undefined; + } + + public prepareColumnNumericStats(profile: ColumnProfile | undefined): void { + this.columnNumericStatsForTable = []; + if (!profile || !profile.statistics) return; + + const stats = profile.statistics; + const dataType = profile.dataType.toLowerCase(); + const numericTypes = ["int", "integer", "float", "double", "numeric", "long"]; + + this.columnNumericStatsForTable.push({ metric: "Null Count", value: stats.nullCount }); + if (isDefined(stats.uniqueCount)) { + this.columnNumericStatsForTable.push({ + metric: "Unique Count", + value: typeof stats.uniqueCount === "number" ? stats.uniqueCount.toLocaleString() : stats.uniqueCount, + }); + } + + if (this.tableProfile && this.tableProfile.globalProfile) { + this.columnNumericStatsForTable.push({ + metric: "Total Rows in Table", + value: this.tableProfile.globalProfile.rowCount.toLocaleString(), + }); + } + + if (numericTypes.includes(dataType)) { + if (isDefined(stats.min)) { + this.columnNumericStatsForTable.push({ + metric: "Min", + value: typeof stats.min === "number" ? stats.min.toLocaleString() : stats.min, + }); + } + if (isDefined(stats.max)) { + this.columnNumericStatsForTable.push({ + metric: "Max", + value: typeof stats.max === "number" ? stats.max.toLocaleString() : stats.max, + }); + } + if (isDefined(stats.mean) && typeof stats.mean === "number") { + this.columnNumericStatsForTable.push({ metric: "Mean", value: stats.mean.toFixed(2) }); + } else if (isDefined(stats.mean)) { + this.columnNumericStatsForTable.push({ metric: "Mean", value: stats.mean }); + } + if (isDefined(stats.stddev) && typeof stats.stddev === "number") { + this.columnNumericStatsForTable.push({ metric: "Std Dev", value: stats.stddev.toFixed(2) }); + } else if (isDefined(stats.stddev)) { + this.columnNumericStatsForTable.push({ metric: "Std Dev", value: stats.stddev }); + } + } + } + + public prepareBarChartData(categoricalCount: { [key: string]: number } | undefined): void { + this.barChartData = []; + if (!categoricalCount) return; + this.barChartData = Object.entries(categoricalCount) + .map(([name, value]) => ({ name, value })) + .sort((a, b) => b.value - a.value) + .slice(0, 10); + } + + private fetchOrGetCachedDataCleaningSuggestions(columnProfile: ColumnProfile): void { + const cached = this.dataCleaningSuggestionsCache.get(columnProfile.columnName); + if (cached) { + this.dataCleaningSuggestions = cached.map(s => ({ ...s, isExpanded: s.isExpanded || false, isAccepting: false })); + this.lastFetchedSuggestionsForColumn = columnProfile.columnName; + this.isLoadingDataCleaningSuggestions = false; + } else { + this.fetchDataCleaningSuggestions(columnProfile); + } + } + + public fetchDataCleaningSuggestions(columnProfile: ColumnProfile | undefined) { + if (!columnProfile || !this.tableProfile || !this.selectedColumnInfo?.operatorId || !this.tableSchema) { + this.dataCleaningSuggestions = []; + this.isLoadingDataCleaningSuggestions = false; + return; + } + + this.isLoadingDataCleaningSuggestions = true; + this.dataCleaningSuggestions = []; + this.workflowSuggestionService + .getDataCleaningSuggestions( + this.selectedColumnInfo.operatorId, + this.tableSchema, + this.tableProfile, + columnProfile.columnName + ) + .pipe( + finalize(() => { + this.isLoadingDataCleaningSuggestions = false; + this.changeDetectorRef.detectChanges(); + }), + untilDestroyed(this) + ) + .subscribe( + (response: WorkflowDataCleaningSuggestionList) => { + this.dataCleaningSuggestions = response.suggestions.map(s => ({ + ...s, + isExpanded: false, + details: s.details, + isAccepting: false, + })); + this.dataCleaningSuggestionsCache.set(columnProfile.columnName, this.dataCleaningSuggestions); + this.lastFetchedSuggestionsForColumn = columnProfile.columnName; + }, + (error: unknown) => { + console.error("Error fetching data cleaning suggestions:", error); + this.dataCleaningSuggestions = []; + } + ); + } + + public refreshDataCleaningSuggestions(): void { + if (this.columnProfile) { + this.dataCleaningSuggestionsCache.delete(this.columnProfile.columnName); + this.fetchDataCleaningSuggestions(this.columnProfile); + } + } + + public toggleSuggestionExpansion(suggestion: DisplayableDataCleaningSuggestion): void { + suggestion.isExpanded = !suggestion.isExpanded; + this.changeDetectorRef.detectChanges(); + } + + public acceptSuggestion(suggestion: DisplayableDataCleaningSuggestion, event: MouseEvent): void { + event.stopPropagation(); + if (!this.selectedColumnInfo || !this.selectedColumnInfo.tableProfile || suggestion.isAccepting) { + if (!this.selectedColumnInfo) console.warn("acceptSuggestion: selectedColumnInfo is null."); + else if (!this.selectedColumnInfo.schema) console.warn("acceptSuggestion: selectedColumnInfo.schema is null."); + else if (suggestion.isAccepting) console.warn("acceptSuggestion: suggestion is already being accepted."); + return; + } + + suggestion.isAccepting = true; + this.messageService.loading(`Attempting to apply action for: "${suggestion.suggestion}"...`, { nzDuration: 0 }); + + const operatorIDToSchema: { [key: string]: ReadonlyArray } = {}; + operatorIDToSchema[this.selectedColumnInfo.operatorId] = this.selectedColumnInfo.schema; + + this.workflowSuggestionService + .getSuggestions( + this.workflowActionService.getWorkflow(), + this.workflowCompilingService.getWorkflowCompilationStateInfo(), + this.executeWorkflowService.getExecutionState(), + suggestion.suggestion, + [this.selectedColumnInfo.operatorId], + operatorIDToSchema + ) + .pipe( + finalize(() => { + suggestion.isAccepting = false; + this.messageService.remove(); + this.changeDetectorRef.detectChanges(); + }), + untilDestroyed(this) + ) + .subscribe( + (actionableSuggestions: WorkflowSuggestionList) => { + if (actionableSuggestions.suggestions && actionableSuggestions.suggestions.length > 0) { + const firstActionableSuggestion = actionableSuggestions.suggestions[0]; + this.messageService.success( + `Applying action: "${firstActionableSuggestion.suggestion}" based on your request.` + ); + this.suggestionActionService.applySuggestion(firstActionableSuggestion); + this.removeSuggestionFromList(suggestion); + } else { + this.messageService.warning( + `Could not find an actionable Texera suggestion for: "${suggestion.suggestion}"` + ); + } + }, + (error: unknown) => { + console.error("Error getting actionable suggestions:", error); + this.messageService.error("Failed to get actionable suggestions."); + } + ); + } + + public rejectSuggestion(suggestion: DisplayableDataCleaningSuggestion, event: MouseEvent): void { + event.stopPropagation(); + this.removeSuggestionFromList(suggestion); + } + + private removeSuggestionFromList(suggestionToRemove: DisplayableDataCleaningSuggestion): void { + this.dataCleaningSuggestions = this.dataCleaningSuggestions.filter( + s => s.suggestionID !== suggestionToRemove.suggestionID + ); + if (this.columnProfile) { + const cached = this.dataCleaningSuggestionsCache.get(this.columnProfile.columnName); + if (cached) { + this.dataCleaningSuggestionsCache.set( + this.columnProfile.columnName, + cached.filter(s => s.suggestionID !== suggestionToRemove.suggestionID) + ); + } + } + this.changeDetectorRef.detectChanges(); + } + + public onChartSelect(event: any): void { + console.log("Chart event:", event); + } +} diff --git a/core/gui/src/app/workspace/component/left-panel/left-panel.component.scss b/core/gui/src/app/workspace/component/left-panel/left-panel.component.scss index 045deb84208..b1e5e6f18a0 100644 --- a/core/gui/src/app/workspace/component/left-panel/left-panel.component.scss +++ b/core/gui/src/app/workspace/component/left-panel/left-panel.component.scss @@ -42,6 +42,9 @@ background: white; width: calc(100% - 33px); z-index: 2; + white-space: nowrap; + overflow: hidden; + line-height: 22px; } #dock { diff --git a/core/gui/src/app/workspace/component/left-panel/left-panel.component.ts b/core/gui/src/app/workspace/component/left-panel/left-panel.component.ts index 6c8f32c4b9e..d88fbb9cafe 100644 --- a/core/gui/src/app/workspace/component/left-panel/left-panel.component.ts +++ b/core/gui/src/app/workspace/component/left-panel/left-panel.component.ts @@ -28,7 +28,10 @@ import { TimeTravelComponent } from "./time-travel/time-travel.component"; import { SettingsComponent } from "./settings/settings.component"; import { calculateTotalTranslate3d } from "../../../common/util/panel-dock"; import { PanelService } from "../../service/panel/panel.service"; +import { ColumnProfileFrameComponent } from "./column-profile-frame/column-profile-frame.component"; +import { ColumnProfileService, SelectedColumnInfo } from "../../service/column-profile/column-profile.service"; import { GuiConfigService } from "../../../common/service/gui-config.service"; + @UntilDestroy() @Component({ selector: "texera-left-panel", @@ -49,7 +52,7 @@ export class LeftPanelComponent implements OnDestroy, OnInit, AfterViewInit { items = [ { component: null, title: "", icon: "", enabled: true }, { component: OperatorMenuComponent, title: "Operators", icon: "appstore", enabled: true }, - { component: VersionsListComponent, title: "Versions", icon: "schedule", enabled: false }, + { component: VersionsListComponent, title: "Versions", icon: "schedule", enabled: environment.userSystemEnabled }, { component: SettingsComponent, title: "Settings", @@ -62,6 +65,12 @@ export class LeftPanelComponent implements OnDestroy, OnInit, AfterViewInit { icon: "history", enabled: false, }, + { + component: ColumnProfileFrameComponent, + title: "Column Profile", + icon: "profile", + enabled: true, + }, { component: TimeTravelComponent, title: "Time Travel", @@ -74,10 +83,12 @@ export class LeftPanelComponent implements OnDestroy, OnInit, AfterViewInit { dragPosition = { x: 0, y: 0 }; returnPosition = { x: 0, y: 0 }; isDocked = true; + public columnProfilePanelIndex = -1; constructor( private panelService: PanelService, - private config: GuiConfigService + private config: GuiConfigService, + private columnProfileService: ColumnProfileService ) { // Initialize items array with config values this.updateItemsWithConfig(); @@ -90,6 +101,31 @@ export class LeftPanelComponent implements OnDestroy, OnInit, AfterViewInit { this.width = Number(localStorage.getItem("left-panel-width")) || this.width; this.height = Number(localStorage.getItem("left-panel-height")) || this.height; + + this.columnProfilePanelIndex = this.items.findIndex(item => item.component === ColumnProfileFrameComponent); + + this.columnProfileService + .getSelectedColumnStream() + .pipe(untilDestroyed(this)) + .subscribe(selectedInfo => { + if (selectedInfo && this.columnProfilePanelIndex !== -1) { + const columnProfileItem = this.items[this.columnProfilePanelIndex]; + if (columnProfileItem) { + columnProfileItem.title = `Profile: ${selectedInfo.columnProfile.columnName}`; + } + this.openFrame(this.columnProfilePanelIndex, true); + } else if (!selectedInfo && this.currentIndex === this.columnProfilePanelIndex) { + const columnProfileItem = this.items[this.columnProfilePanelIndex]; + if (columnProfileItem) { + columnProfileItem.title = "Column Profile"; + } + } else if (this.currentIndex === this.columnProfilePanelIndex && selectedInfo === null) { + const columnProfileItem = this.items[this.columnProfilePanelIndex]; + if (columnProfileItem) { + columnProfileItem.title = "Column Profile"; + } + } + }); } private updateItemsWithConfig(): void { @@ -145,17 +181,26 @@ export class LeftPanelComponent implements OnDestroy, OnInit, AfterViewInit { } } - openFrame(i: number) { - if (!i) { + openFrame(i: number, forceOpen: boolean = false) { + if (i === 0) { this.width = 0; this.height = 65; - } else if (!this.width) { - this.width = LeftPanelComponent.MIN_PANEL_WIDTH; - this.height = this.minPanelHeight; + } else { + if (this.width === 0 || (forceOpen && this.width < LeftPanelComponent.MIN_PANEL_WIDTH)) { + this.width = LeftPanelComponent.MIN_PANEL_WIDTH; + this.height = this.minPanelHeight; + } + } + + if (i >= 0 && i < this.items.length) { + if (i === this.columnProfilePanelIndex && this.items[i].title.startsWith("Profile: ")) { + this.title = this.items[i].title; + } else { + this.title = this.items[i].title; + } + this.currentComponent = this.items[i].component; + this.currentIndex = i; } - this.title = this.items[i].title; - this.currentComponent = this.items[i].component; - this.currentIndex = i; } onDrop(event: CdkDragDrop) { moveItemInArray(this.order, event.previousIndex, event.currentIndex); diff --git a/core/gui/src/app/workspace/component/power-button/computing-unit-selection.component.html b/core/gui/src/app/workspace/component/power-button/computing-unit-selection.component.html index ecd70042b7c..8c299085b00 100644 --- a/core/gui/src/app/workspace/component/power-button/computing-unit-selection.component.html +++ b/core/gui/src/app/workspace/component/power-button/computing-unit-selection.component.html @@ -137,6 +137,7 @@ nz-menu-divider>
  • + +
    + + +
    + + +
    + + Powered by Data Copilot +
    +
    = new Subject(); + /** + * Text typed by user that describes the intention for the highlighted operator. It will be sent + * to the backend suggestion service when the user clicks the "Fill" button. + */ + intentionText: string = ""; + + /** Whether we are currently waiting for the backend to return suggestions. */ + loadingSuggestions = false; + + /** Whether the Copilot intention box is visible */ + copilotEnabled: boolean = false; + + public selectedColumnInfo: SelectedColumnInfo | null = null; + constructor( private formlyJsonschema: FormlyJsonschema, + private columnProfileService: ColumnProfileService, private workflowActionService: WorkflowActionService, public executeWorkflowService: ExecuteWorkflowService, private dynamicSchemaService: DynamicSchemaService, @@ -155,7 +173,9 @@ export class OperatorPropertyEditFrameComponent implements OnInit, OnChanges, On private changeDetectorRef: ChangeDetectorRef, private workflowVersionService: WorkflowVersionService, private workflowStatusSerivce: WorkflowStatusService, - private config: GuiConfigService + private workflowSuggestionService: WorkflowSuggestionService, + private suggestionActionService: SuggestionActionService, + private config: GuiConfigService, ) {} ngOnChanges(changes: SimpleChanges): void { @@ -193,6 +213,13 @@ export class OperatorPropertyEditFrameComponent implements OnInit, OnChanges, On this.currentOperatorStatus = update[this.currentOperatorId]; } }); + + this.columnProfileService + .getSelectedColumnStream() + .pipe(untilDestroyed(this)) + .subscribe(selectedInfo => { + this.selectedColumnInfo = selectedInfo; + }); } async ngOnDestroy() { @@ -783,4 +810,63 @@ export class OperatorPropertyEditFrameComponent implements OnInit, OnChanges, On theme: "snow", }); } + + /** + * Trigger the workflow suggestion service using the user-provided intention text. The + * SuggestionFrameComponent is subscribed to the global suggestion stream and will update + * automatically when the result comes back. + */ + fillSuggestions(): void { + if (this.loadingSuggestions) { + return; + } + + const operators = this.workflowActionService.getTexeraGraph().getAllOperators(); + if (operators.length === 0) { + return; + } + + const highlighted = this.workflowActionService.getJointGraphWrapper().getCurrentHighlightedOperatorIDs(); + if (highlighted.length === 0) { + this.notificationService.warning("No operator selected."); + return; + } + + const operatorIDToSchema: { [key: string]: ReadonlyArray } = {}; + if (this.selectedColumnInfo) { + operatorIDToSchema[this.selectedColumnInfo.operatorId] = this.selectedColumnInfo.schema; + } + + this.notificationService.info("Asking copilot to fill out the properties..."); + this.loadingSuggestions = true; + + this.workflowSuggestionService + .getSuggestions( + this.workflowActionService.getWorkflow(), + this.workflowCompilingService.getWorkflowCompilationStateInfo(), + this.executeWorkflowService.getExecutionState(), + this.intentionText.trim(), + highlighted, + operatorIDToSchema + ) + .pipe(finalize(() => { + this.loadingSuggestions = false; + this.notificationService.remove(); + })) + .subscribe({ + next: suggestionsList => { + const suggestions = suggestionsList.suggestions; + if (suggestions.length > 0) { + const suggestion = suggestions[0]; + this.notificationService.success(`Applying suggestion: "${suggestion.suggestion}"`); + this.suggestionActionService.applySuggestion(suggestion); + } else { + this.notificationService.warning("No actionable suggestion was found."); + } + }, + error: unknown => { + this.notificationService.error("Failed to get actionable suggestions."); + }, + }); + } } diff --git a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.html b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.html index 374777cd52e..dc282452d73 100644 --- a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.html +++ b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.html @@ -45,6 +45,13 @@

    No error to display.

    nzType="aim" title="focus operator" (click)="$event.stopPropagation(); onClickGotoButton(error.operatorId)"> +

    {{ error.details }}

    diff --git a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.scss b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.scss index 0c3aa937acb..06fe71e075d 100644 --- a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.scss +++ b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.scss @@ -50,3 +50,13 @@ .error-category { font-weight: bold; } + +.fix-error-icon { + color: #1890ff; + cursor: pointer; + transition: transform 0.3s ease; +} + +.fix-error-icon:hover { + transform: rotate(180deg) scale(1.2); +} diff --git a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.ts b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.ts index 6e8994803ae..16b5ba4d280 100644 --- a/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.ts +++ b/core/gui/src/app/workspace/component/result-panel/error-frame/error-frame.component.ts @@ -20,12 +20,14 @@ import { Component, ElementRef, Input, OnChanges, OnInit, SimpleChanges, ViewChild } from "@angular/core"; import { ExecuteWorkflowService } from "../../../service/execute-workflow/execute-workflow.service"; import { WorkflowConsoleService } from "../../../service/workflow-console/workflow-console.service"; -import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; import { WorkflowWebsocketService } from "../../../service/workflow-websocket/workflow-websocket.service"; import { WorkflowFatalError } from "../../../types/workflow-websocket.interface"; import { render } from "sass"; import { WorkflowActionService } from "../../../service/workflow-graph/model/workflow-action.service"; import { WorkflowCompilingService } from "../../../service/compile-workflow/workflow-compiling.service"; +import { WorkflowSuggestionService } from "../../../service/workflow-suggestion/workflow-suggestion.service"; +import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; +import { NotificationService } from "src/app/common/service/notification/notification.service"; @UntilDestroy() @Component({ @@ -38,14 +40,25 @@ export class ErrorFrameComponent implements OnInit { // display error message: categoryToErrorMapping: ReadonlyMap> = new Map(); + // Whether suggestion service is currently loading + isLoading = false; + constructor( private executeWorkflowService: ExecuteWorkflowService, private workflowActionService: WorkflowActionService, - private workflowCompilingService: WorkflowCompilingService + private workflowCompilingService: WorkflowCompilingService, + private workflowSuggestionService: WorkflowSuggestionService, + private notificationService: NotificationService ) {} ngOnInit(): void { this.renderError(); + + // Listen to loading state to spin icon + this.workflowSuggestionService + .getLoadingStream() + .pipe(untilDestroyed(this)) + .subscribe(v => (this.isLoading = v)); } onClickGotoButton(target: string) { @@ -70,4 +83,34 @@ export class ErrorFrameComponent implements OnInit { return acc; }, new Map()); } + + /** + * Ask workflow copilot to provide a fix for this error. + */ + onClickFixError(error: WorkflowFatalError) { + const focusingIDs = error.operatorId && error.operatorId !== "unknown operator" ? [error.operatorId] : []; + this.notificationService.info("Asking copilot to fix this error..."); + // Fire suggestion request + this.workflowSuggestionService + .getSuggestions( + this.workflowActionService.getWorkflow(), + this.workflowCompilingService.getWorkflowCompilationStateInfo(), + this.executeWorkflowService.getExecutionState(), + "Please fix this error", + focusingIDs, + {} + ) + .pipe(untilDestroyed(this)) + .subscribe(_ => { + this.notificationService.success("Received fix suggestions from the copilot"); + }); + + // Switch to Suggestions tab so the user sees loading spinner + setTimeout(() => { + const suggestionTab = document.querySelector(".ant-tabs-tab[aria-controls*=\"Suggestions\"]") as HTMLElement; + if (suggestionTab) { + suggestionTab.click(); + } + }, 50); + } } diff --git a/core/gui/src/app/workspace/component/result-panel/result-panel.component.ts b/core/gui/src/app/workspace/component/result-panel/result-panel.component.ts index 98708cf0763..ee60a158044 100644 --- a/core/gui/src/app/workspace/component/result-panel/result-panel.component.ts +++ b/core/gui/src/app/workspace/component/result-panel/result-panel.component.ts @@ -51,6 +51,7 @@ import { PanelService } from "../../service/panel/panel.service"; import { WorkflowCompilingService } from "../../service/compile-workflow/workflow-compiling.service"; import { CompilationState } from "../../types/workflow-compiling.interface"; import { WorkflowFatalError } from "../../types/workflow-websocket.interface"; +import { SuggestionFrameComponent } from "./suggestion-frame/suggestion-frame.component"; export const DEFAULT_WIDTH = 800; export const DEFAULT_HEIGHT = 300; @@ -99,6 +100,9 @@ export class ResultPanelComponent implements OnInit, OnDestroy { } ngOnInit(): void { + // Add suggestions tab to the result panel first + // this.displaySuggestions(); + const style = localStorage.getItem("result-panel-style"); if (style) document.getElementById("result-container")!.style.cssText = style; const translates = document.getElementById("result-container")!.style.transform; @@ -218,14 +222,19 @@ export class ResultPanelComponent implements OnInit, OnDestroy { const currentHighlightedOperator = highlightedOperators.length === 1 ? highlightedOperators[0] : undefined; if (this.currentOperatorId !== currentHighlightedOperator) { - // clear everything, prepare for state change - this.clearResultPanel(); + // clear result-related panels, but keep the suggestions tab + this.clearResultPanelExceptSuggestions(); this.currentOperatorId = currentHighlightedOperator; if (!this.currentOperatorId) { this.operatorTitle = ""; } } + // + // // Make sure suggestions tab is always available + // if (!this.frameComponentConfigs.has("Suggestions")) { + // this.displaySuggestions(); + // } if ( this.executeWorkflowService.getExecutionState().state === ExecutionState.Failed || @@ -258,6 +267,22 @@ export class ResultPanelComponent implements OnInit, OnDestroy { this.frameComponentConfigs.clear(); } + /** + * Clears result panel components except for the Suggestions tab + */ + clearResultPanelExceptSuggestions(): void { + // Temporarily store the suggestions component if it exists + const suggestionsComponent = this.frameComponentConfigs.get("Suggestions"); + + // Clear all components + this.frameComponentConfigs.clear(); + + // Restore the suggestions component if it existed + if (suggestionsComponent) { + this.frameComponentConfigs.set("Suggestions", suggestionsComponent); + } + } + displayConsole(operatorId: string, consoleInputEnabled: boolean) { this.frameComponentConfigs.set("Console", { component: ConsoleFrameComponent, @@ -339,11 +364,19 @@ export class ResultPanelComponent implements OnInit, OnDestroy { openPanel() { this.height = DEFAULT_HEIGHT; this.width = DEFAULT_WIDTH; + + // // Ensure suggestions tab is available when panel is opened + // if (!this.frameComponentConfigs.has("Suggestions")) { + // this.displaySuggestions(); + // } } closePanel() { this.height = 32.5; this.width = 0; + + // Don't clear suggestions when closing the panel + // Preserve the tab for when the panel is reopened } resetPanelPosition() { @@ -390,4 +423,14 @@ export class ResultPanelComponent implements OnInit, OnDestroy { if (!isDefined(newHeight)) return; this.returnPosition = { x: this.returnPosition.x, y: this.returnPosition.y + prevHeight - newHeight }; } + + // /** + // * Displays the workflow suggestions in the result panel + // */ + // displaySuggestions() { + // this.frameComponentConfigs.set("Suggestions", { + // component: SuggestionFrameComponent, + // componentInputs: {}, + // }); + // } } diff --git a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.html b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.html index 2f13bd6984d..6b1f26e3b10 100644 --- a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.html +++ b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.html @@ -25,6 +25,20 @@

    Empty result set

    +
    Empty result set ngClass="header-size" style="text-align: center" nzWidth="widthPercent"> - {{ column.header }} - - - - -
    - -
    -
    Min
    -
    - -
    -
    -
    - -
    -
    Max
    -
    - -
    -
    -
    - -
    -
    Non-Null Count
    -
    - -
    -
    -
    - -
    -
    - {{tableStats[column.header]['firstCat']}} - (approximate) -
    -
    %
    -
    -
    - -
    -
    - {{tableStats[column.header]['secondCat']}} - (approximate) -
    -
    %
    -
    -
    - -
    -
    - Other - (approximate) -
    -
    %
    -
    -
    +
    + {{ column.header }}
    @@ -143,3 +96,43 @@
    + + + +
    + + {{ gp.rowCount.toLocaleString() }} + {{ gp.columnCount }} + {{ gp.samplesUsed.toLocaleString() }} + {{ (gp.uniqueRowRatio * 100).toFixed(2) }}% + {{ gp.duplicateRowCount.toLocaleString() }} + {{ (gp.rowHasNullRatio * 100).toFixed(2) }}% + {{ (gp.rowIsNullRatio * 100).toFixed(2) }}% + {{ gp.fileType }} + {{ gp.encoding }} + + {{ gp.times.rowStatsMs.toFixed(2) }} + + +
    +
    diff --git a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.scss b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.scss index 0ab27a8530e..b3720ca30ab 100644 --- a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.scss +++ b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.scss @@ -135,3 +135,139 @@ th.header-size { bottom: 16px; right: 0; } + +.table-controls { + display: flex; + justify-content: flex-end; + margin-bottom: 10px; +} + +.global-stats-btn { + display: flex; + align-items: center; + gap: 5px; +} + +.column-header { + display: flex; + align-items: center; + justify-content: center; + cursor: pointer; + padding: 8px 4px; + transition: + background-color 0.2s ease-in-out, + color 0.2s ease-in-out; + + &:hover { + background-color: #e6f7ff; + color: #096dd9; + } +} + +.custom-profile-row { + background-color: #f9f9f9; + + .profile-header { + font-size: 12px; + padding: 4px; + } + + .profile-line { + display: flex; + justify-content: space-between; + margin-bottom: 2px; + } + + .profile-label { + font-weight: 500; + color: #555; + } + + .profile-value { + font-weight: normal; + } +} + +// Styles for the column details modal +:host ::ng-deep .column-details, +:host ::ng-deep .global-stats { + .detail-row, + .stat-row { + display: flex; + margin-bottom: 8px; + + .detail-label, + .stat-label { + font-weight: 500; + width: 40%; + color: #555; + } + + .detail-value, + .stat-value { + width: 60%; + } + } +} + +// You might want to add specific styles for the modal content if needed, +// but ng-zorro components usually come with good defaults. +// For example, to ensure ngx-charts is responsive within the modal: +:host ::ng-deep .ngx-charts { + display: block; + width: 100% !important; // Override default fixed width if necessary + height: auto; +} + +:host ::ng-deep .chart-legend .legend-labels { + // If legend is enabled and overlaps + white-space: normal !important; +} + +// Styles for Data Cleaning Suggestions +.data-cleaning-suggestions-section { + .no-suggestions-message { + padding: 10px; + background-color: #f0f0f0; // A neutral background for no suggestions + border: 1px solid #d9d9d9; + border-radius: 4px; + text-align: center; + color: #555; + } + + .suggestions-list { + // Changed from ul to .suggestions-list for more specific styling + background-color: #fffbe6; + border: 1px solid #ffe58f; + padding: 10px; // Adjusted padding + border-radius: 4px; + list-style-type: none; // Remove default bullets, will use custom or none + margin: 0; + + .suggestion-item { + padding: 8px 12px; + margin-bottom: 5px; + color: #333; + border-radius: 3px; + cursor: pointer; + transition: + background-color 0.2s ease-in-out, + color 0.2s ease-in-out; + + &:last-child { + margin-bottom: 0; + } + + &:hover { + background-color: #ffe58f; // Darker yellow on hover + color: #000; + } + + .suggestion-type { + font-size: 0.9em; + color: #888; + margin-left: 5px; + } + } + } +} diff --git a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.ts b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.ts index 7c3bc84bf6d..48b9a5b70ae 100644 --- a/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.ts +++ b/core/gui/src/app/workspace/component/result-panel/result-table-frame/result-table-frame.component.ts @@ -17,7 +17,7 @@ * under the License. */ -import { Component, Input, OnChanges, OnInit, SimpleChanges } from "@angular/core"; +import { Component, Input, OnChanges, OnInit, SimpleChanges, ViewChild, TemplateRef } from "@angular/core"; import { NzModalRef, NzModalService } from "ng-zorro-antd/modal"; import { NzTableQueryParams } from "ng-zorro-antd/table"; import { WorkflowActionService } from "../../../service/workflow-graph/model/workflow-action.service"; @@ -27,10 +27,25 @@ import { isWebPaginationUpdate } from "../../../types/execute-workflow.interface import { IndexableObject, TableColumn } from "../../../types/result-table.interface"; import { RowModalComponent } from "../result-panel-modal.component"; import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; -import { DomSanitizer, SafeHtml } from "@angular/platform-browser"; import { ResultExportationComponent } from "../../result-exportation/result-exportation.component"; import { ChangeDetectorRef } from "@angular/core"; import { SchemaAttribute } from "../../../types/workflow-compiling.interface"; +import { WorkflowStatusService } from "../../../service/workflow-status/workflow-status.service"; +import { + TableProfile, + ColumnProfile, + ColumnStatistics, +} from "../../../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; +import { WorkflowSuggestionService } from "../../../service/workflow-suggestion/workflow-suggestion.service"; +import { finalize } from "rxjs"; +import { + WorkflowDataCleaningSuggestion, + WorkflowDataCleaningSuggestionList, + WorkflowSuggestion, + WorkflowSuggestionList, +} from "../../../types/workflow-suggestion.interface"; +import { isDefined } from "../../../../common/util/predicate"; +import { ColumnProfileService } from "../../../service/column-profile/column-profile.service"; /** * The Component will display the result in an excel table format, @@ -66,19 +81,23 @@ export class ResultTableFrameComponent implements OnInit, OnChanges { totalNumTuples: number = 0; pageSize = 5; panelHeight = 0; - tableStats: Record> = {}; - prevTableStats: Record> = {}; widthPercent: string = ""; sinkStorageMode: string = ""; private schema: ReadonlyArray = []; + tableProfile: TableProfile | undefined; + + // For Global Stats Modal + @ViewChild("globalStatsModalContent") globalStatsModalContent!: TemplateRef; constructor( private modalService: NzModalService, private workflowActionService: WorkflowActionService, private workflowResultService: WorkflowResultService, private resizeService: PanelResizeService, - private sanitizer: DomSanitizer, - private changeDetectorRef: ChangeDetectorRef + private changeDetectorRef: ChangeDetectorRef, + private workflowStatusService: WorkflowStatusService, + private workflowSuggestionService: WorkflowSuggestionService, + private columnProfileService: ColumnProfileService ) {} ngOnChanges(changes: SimpleChanges): void { @@ -90,11 +109,9 @@ export class ResultTableFrameComponent implements OnInit, OnChanges { this.totalNumTuples = paginatedResultService.getCurrentTotalNumTuples(); this.currentPageIndex = paginatedResultService.getCurrentPageIndex(); this.changePaginatedResultData(); - - this.tableStats = paginatedResultService.getStats(); - this.prevTableStats = this.tableStats; this.schema = paginatedResultService.getSchema(); } + this.subscribeToTableProfile(); } } @@ -120,24 +137,6 @@ export class ResultTableFrameComponent implements OnInit, OnChanges { this.changeDetectorRef.detectChanges(); }); - this.workflowResultService - .getResultTableStats() - .pipe(untilDestroyed(this)) - .subscribe(([prevStats, currentStats]) => { - if (!this.operatorId) { - return; - } - - if (currentStats[this.operatorId]) { - this.tableStats = currentStats[this.operatorId]; - if (prevStats[this.operatorId] && this.checkKeys(this.tableStats, prevStats[this.operatorId])) { - this.prevTableStats = prevStats[this.operatorId]; - } else { - this.prevTableStats = this.tableStats; - } - } - }); - this.workflowResultService .getSinkStorageMode() .pipe(untilDestroyed(this)) @@ -161,55 +160,10 @@ export class ResultTableFrameComponent implements OnInit, OnChanges { this.schema = paginatedResultService.getSchema(); } } - } - - checkKeys( - currentStats: Record>, - prevStats: Record> - ): boolean { - let firstSet = Object.keys(currentStats); - let secondSet = Object.keys(prevStats); - - if (firstSet.length != secondSet.length) { - return false; - } - - for (let i = 0; i < firstSet.length; i++) { - if (firstSet[i] != secondSet[i]) { - return false; - } - } - - return true; - } - - compare(field: string, stats: string): SafeHtml { - let current = this.tableStats[field][stats]; - let previous = this.prevTableStats[field][stats]; - let currentStr = ""; - let previousStr = ""; - - if (typeof current === "number" && typeof previous === "number") { - currentStr = current.toFixed(2); - previousStr = previous !== undefined ? previous.toFixed(2) : currentStr; - } else { - currentStr = current.toLocaleString(); - previousStr = previous !== undefined ? previous.toLocaleString() : currentStr; - } - let styledValue = ""; - for (let i = 0; i < currentStr.length; i++) { - const char = currentStr[i]; - const prevChar = previousStr[i]; - - if (char !== prevChar) { - styledValue += `${char}`; - } else { - styledValue += `${char}`; - } + if (this.operatorId) { + this.subscribeToTableProfile(); } - - return this.sanitizer.bypassSecurityTrustHtml(styledValue); } private adjustPageSizeBasedOnPanelSize(panelHeight: number) { @@ -386,4 +340,85 @@ export class ResultTableFrameComponent implements OnInit, OnChanges { nzFooter: null, }); } + + private subscribeToTableProfile(): void { + if (!this.operatorId) { + return; + } + + // 1. set existing cached profile (if any) + const cached = this.workflowStatusService.getCurrentTableProfiles(); + if (cached && cached[this.operatorId]) { + this.tableProfile = cached[this.operatorId]; + } + // 2. listen to subsequent updates + this.workflowStatusService + .getTableProfilesUpdateStream() + .pipe(untilDestroyed(this)) + .subscribe(profiles => { + const prof = profiles[this.operatorId!]; + if (prof) { + this.tableProfile = prof; + this.changeDetectorRef.detectChanges(); + } + }); + } + + getColumnProfile(columnName: string): ColumnProfile | undefined { + if (!this.tableProfile || !this.tableProfile.columnProfiles) return undefined; + + const target = columnName.trim(); + + // exact match + let profile = this.tableProfile.columnProfiles.find(p => p.columnName.trim() === target); + + // case-insensitive fallback + if (!profile) { + profile = this.tableProfile.columnProfiles.find(p => p.columnName.trim().toLowerCase() === target.toLowerCase()); + } + + return profile; + } + + showColumnDetails(columnName: string, event: MouseEvent): void { + event.stopPropagation(); + if (!this.operatorId || !this.tableProfile) { + console.warn("OperatorId or TableProfile is not available to show column details."); + return; + } + + const columnProfile = this.getColumnProfile(columnName); + if (!columnProfile) { + console.warn(`Could not find profile for column: ${columnName}`); + return; + } + + // Announce the selected column, now including the schema + this.columnProfileService.selectColumn({ + operatorId: this.operatorId, + columnProfile: columnProfile, + tableProfile: this.tableProfile, + schema: this.schema, + }); + + // The LeftPanelComponent is responsible for opening the correct frame + // when it detects a change from columnProfileService. + } + + showGlobalStats(): void { + if (!this.tableProfile || !this.tableProfile.globalProfile) return; + + this.modalService.create({ + nzTitle: "Table Statistics", + nzContent: this.globalStatsModalContent, + nzWidth: 600, + nzFooter: [ + { + label: "OK", + type: "primary", + onClick: () => this.modalService.closeAll(), + }, + ], + }); + } } diff --git a/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.html b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.html new file mode 100644 index 00000000000..70e53a76d6b --- /dev/null +++ b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.html @@ -0,0 +1,96 @@ +
    +
    +
    +

    Workflow Copilot

    +

    Click on a suggestion to preview

    +
    + +
    + +
    + +
    +
    + +
    + +
    +
    + {{suggestion.suggestion}} +
    + +
    + + + +
    +
    + +
    + No suggestions available +
    +
    + +
    + +
    +
    +
    diff --git a/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.scss b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.scss new file mode 100644 index 00000000000..9384b2d94b1 --- /dev/null +++ b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.scss @@ -0,0 +1,149 @@ +.suggestion-container { + display: flex; + flex-direction: column; + height: 100%; + width: 100%; + overflow: hidden; +} + +.header { + padding: 10px; + border-bottom: 1px solid #ddd; + + h3 { + margin-bottom: 5px; + } + + p { + color: #666; + margin-bottom: 0; + font-size: 12px; + } +} + +.suggestions-list { + flex: 1; + overflow-y: auto; + padding: 10px; +} + +.loading-container { + display: flex; + justify-content: center; + align-items: center; + min-height: 200px; +} + +.suggestion-item { + border: 1px solid #ddd; + border-radius: 4px; + margin-bottom: 10px; + transition: all 0.3s; + + &:hover { + border-color: #1890ff; + box-shadow: 0 2px 8px rgba(0, 0, 0, 0.09); + } + + &.active { + border-color: #1890ff; + box-shadow: 0 2px 8px rgba(0, 0, 0, 0.15); + + .suggestion-content { + background-color: #e6f7ff; + } + } +} + +.suggestion-content { + padding: 10px; + cursor: pointer; + transition: background-color 0.3s; + + .description { + display: block; + word-break: break-word; + } +} + +.suggestion-actions { + display: flex; + padding: 5px 10px 10px; + flex-wrap: wrap; + gap: 8px; + justify-content: space-between; + + button { + flex: 1; + min-width: 80px; + margin: 0; + z-index: 100; /* Ensure buttons are on top */ + + /* Fix for specificity */ + &[nz-button] { + margin: 0; + padding: 0 8px; + height: 32px; + line-height: 30px; + } + + &:nth-child(1) { + // Apply button + background-color: #52c41a; + border-color: #52c41a; + color: white; + + &:hover { + background-color: #73d13d; + border-color: #73d13d; + } + } + + &:nth-child(2) { + // Dislike button + color: #f5222d; + border-color: #f5222d; + + &:hover { + background-color: #fff1f0; + } + } + + &:nth-child(3) { + // Cancel button + color: #595959; + border-color: #d9d9d9; + + &:hover { + color: #1890ff; + border-color: #1890ff; + } + } + } +} + +.no-suggestions { + text-align: center; + color: #999; + padding: 20px; +} + +// Style for bottom refresh button +.suggestion-container > .suggestion-actions { + border-top: 1px solid #ddd; + padding: 10px; + justify-content: center; +} + +// Style for the container when preview is active +#suggestion-frame-container { + height: 100%; + width: 100%; + overflow: hidden; + position: relative; + + &.preview-active { + // Make sure the suggestion tab takes priority + z-index: 1000; + } +} diff --git a/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.ts b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.ts new file mode 100644 index 00000000000..14eaea9ec23 --- /dev/null +++ b/core/gui/src/app/workspace/component/result-panel/suggestion-frame/suggestion-frame.component.ts @@ -0,0 +1,206 @@ +import { Component, OnInit, OnDestroy, ChangeDetectorRef, NgZone } from "@angular/core"; +import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; +import { WorkflowActionService } from "../../../service/workflow-graph/model/workflow-action.service"; +import { WorkflowSuggestionService } from "../../../service/workflow-suggestion/workflow-suggestion.service"; +import { NzMessageService } from "ng-zorro-antd/message"; +import { ExecuteWorkflowService } from "../../../service/execute-workflow/execute-workflow.service"; +import { WorkflowCompilingService } from "../../../service/compile-workflow/workflow-compiling.service"; +import { Subscription, interval, Observable } from "rxjs"; +import { WorkflowSuggestion, WorkflowSuggestionList } from "../../../types/workflow-suggestion.interface"; +import { SuggestionActionService } from "../../../service/suggestion-action/suggestion-action.service"; + +/** + * SuggestionFrameComponent is a wrapper for the workflow suggestion functionality + * that allows it to be displayed in the result panel as a tab. + */ +@UntilDestroy() +@Component({ + selector: "texera-suggestion-frame", + templateUrl: "./suggestion-frame.component.html", + styleUrls: ["./suggestion-frame.component.scss"], +}) +export class SuggestionFrameComponent implements OnInit, OnDestroy { + public suggestions: WorkflowSuggestion[] = []; + public canModify = true; + public loadingSuggestions = false; + public intentionText = ""; + + // Observables from SuggestionActionService for template binding + public activePreviewId: string | null = null; + public isInPreviewMode: boolean = false; + + private tabFocusInterval: Subscription | null = null; + private boundHandleDocumentClick: any; + + constructor( + private workflowActionService: WorkflowActionService, + private workflowCompilingService: WorkflowCompilingService, + private workflowExecuteService: ExecuteWorkflowService, + private messageService: NzMessageService, + private workflowSuggestionService: WorkflowSuggestionService, + private suggestionActionService: SuggestionActionService, + private cdr: ChangeDetectorRef, + private ngZone: NgZone + ) { + this.suggestionActionService.activePreviewId$.pipe(untilDestroyed(this)).subscribe(id => { + this.activePreviewId = id; + this.cdr.detectChanges(); + }); + + this.suggestionActionService.isInPreviewMode$.pipe(untilDestroyed(this)).subscribe(inPreview => { + this.isInPreviewMode = inPreview; + if (inPreview) { + this.addDocumentClickListener(); + } else { + this.removeDocumentClickListener(); + } + this.cdr.detectChanges(); + }); + this.boundHandleDocumentClick = this.handleDocumentClick.bind(this); + } + + ngOnInit(): void { + this.workflowActionService + .getWorkflowModificationEnabledStream() + .pipe(untilDestroyed(this)) + .subscribe(canModify => (this.canModify = canModify)); + + this.workflowSuggestionService + .getSuggestionsListStream() + .pipe(untilDestroyed(this)) + .subscribe(list => { + this.suggestions = list.suggestions; + this.cdr.detectChanges(); + }); + + this.workflowSuggestionService + .getLoadingStream() + .pipe(untilDestroyed(this)) + .subscribe(isLoading => { + this.loadingSuggestions = isLoading; + this.cdr.detectChanges(); + }); + } + + ngOnDestroy(): void { + this.removeDocumentClickListener(); + + if (this.tabFocusInterval) { + this.tabFocusInterval.unsubscribe(); + this.tabFocusInterval = null; + } + } + + private addDocumentClickListener(): void { + if (!document.hasOwnProperty("suggestionsClickListener")) { + document.addEventListener("click", this.boundHandleDocumentClick, true); + // @ts-ignore + document.suggestionsClickListener = true; + } + } + + private removeDocumentClickListener(): void { + document.removeEventListener("click", this.boundHandleDocumentClick, true); + // @ts-ignore + delete document.suggestionsClickListener; + } + + private handleDocumentClick(event: MouseEvent): void { + if (this.isInPreviewMode) { + const actionButton = (event.target as HTMLElement).closest(".suggestion-actions button"); + const target = event.target as HTMLElement; + + if (actionButton) return; + + if (target && (target.closest(".ant-tabs-tab") || target.closest("a[href]"))) { + if (!target.closest("texera-suggestion-frame")) { + event.preventDefault(); + event.stopPropagation(); + this.messageService.warning("Please cancel the preview first before changing tabs"); + this.focusSuggestionTab(); + } + } + } + } + + public refreshSuggestions(): void { + if (this.isInPreviewMode) { + this.messageService.info("Please cancel the active preview before refreshing suggestions."); + return; + } + const operators = this.workflowActionService.getTexeraGraph().getAllOperators(); + if (operators.length === 0) { + this.messageService.info("Cannot generate suggestions for an empty workflow."); + return; + } + const focusedOperatorIDs = this.workflowActionService.getJointGraphWrapper().getCurrentHighlightedOperatorIDs(); + const intention = this.intentionText.trim(); + + this.workflowSuggestionService + .getSuggestions( + this.workflowActionService.getWorkflow(), + this.workflowCompilingService.getWorkflowCompilationStateInfo(), + this.workflowExecuteService.getExecutionState(), + this.intentionText.trim(), + this.workflowActionService.getJointGraphWrapper().getCurrentHighlightedOperatorIDs(), + {} + ) + .pipe(untilDestroyed(this)) + .subscribe(); + } + + public togglePreview(suggestion: WorkflowSuggestion): void { + this.suggestionActionService.togglePreview(suggestion); + if (this.isInPreviewMode) { + this.focusSuggestionTab(); + } + } + + private focusSuggestionTab(): void { + setTimeout(() => { + const suggestionTab = document.querySelector(".ant-tabs-tab[aria-controls*=\"Suggestions\"]") as HTMLElement; + if (suggestionTab) { + suggestionTab.click(); + if (this.isInPreviewMode) { + if (this.tabFocusInterval) this.tabFocusInterval.unsubscribe(); + this.tabFocusInterval = interval(300) + .pipe(untilDestroyed(this)) + .subscribe(() => { + if (this.isInPreviewMode) { + const activeTab = document.querySelector(".ant-tabs-tab-active"); + const isSuggestionTabActive = + activeTab && + (activeTab.textContent?.includes("Suggestions") || + activeTab.getAttribute("aria-controls")?.includes("Suggestions")); + if (!isSuggestionTabActive) { + const currentSuggestionTab = document.querySelector( + ".ant-tabs-tab[aria-controls*=\"Suggestions\"]" + ) as HTMLElement; + if (currentSuggestionTab) currentSuggestionTab.click(); + } + } else if (this.tabFocusInterval) { + this.tabFocusInterval.unsubscribe(); + this.tabFocusInterval = null; + } + }); + } + } + }, 50); + } + + public dislikeSuggestion(suggestion: WorkflowSuggestion): void { + if (this.activePreviewId === suggestion.suggestionID) { + this.suggestionActionService.clearPreviewAndRestoreWorkflow(); + } + this.workflowSuggestionService.removeSuggestionById(suggestion.suggestionID); + this.messageService.info("Suggestion removed from the list."); + } + + public cancelPreview(): void { + this.suggestionActionService.clearPreviewAndRestoreWorkflow(); + } + + public applySuggestion(suggestion: WorkflowSuggestion): void { + this.suggestionActionService.applySuggestion(suggestion); + } +} diff --git a/core/gui/src/app/workspace/service/column-profile/column-profile.service.ts b/core/gui/src/app/workspace/service/column-profile/column-profile.service.ts new file mode 100644 index 00000000000..5d521fd26d6 --- /dev/null +++ b/core/gui/src/app/workspace/service/column-profile/column-profile.service.ts @@ -0,0 +1,45 @@ +import { Injectable } from "@angular/core"; +import { BehaviorSubject, Observable } from "rxjs"; +import { + ColumnProfile, + TableProfile, +} from "../../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; +import { SchemaAttribute } from "../../types/workflow-compiling.interface"; + +export interface SelectedColumnInfo { + operatorId: string; + columnProfile: ColumnProfile; + tableProfile: TableProfile; // Pass the whole table profile for context (e.g. global row count) + schema: ReadonlyArray; +} + +@Injectable({ + providedIn: "root", +}) +export class ColumnProfileService { + private selectedColumnSubject = new BehaviorSubject(null); + + constructor() {} + + /** + * Call this method to select a column for profiling. + * This will notify all subscribers. + */ + public selectColumn(info: SelectedColumnInfo | null): void { + this.selectedColumnSubject.next(info); + } + + /** + * Get an observable stream of the currently selected column for profiling. + */ + public getSelectedColumnStream(): Observable { + return this.selectedColumnSubject.asObservable(); + } + + /** + * Clears the currently selected column. + */ + public clearColumnSelection(): void { + this.selectedColumnSubject.next(null); + } +} diff --git a/core/gui/src/app/workspace/service/compile-workflow/workflow-compiling.service.ts b/core/gui/src/app/workspace/service/compile-workflow/workflow-compiling.service.ts index a82a8fdd597..55e8baa5fbd 100644 --- a/core/gui/src/app/workspace/service/compile-workflow/workflow-compiling.service.ts +++ b/core/gui/src/app/workspace/service/compile-workflow/workflow-compiling.service.ts @@ -123,6 +123,16 @@ export class WorkflowCompilingService { return this.currentCompilationStateInfo.state; } + public getOperatorInputSchemaMap(): Record { + if ( + this.currentCompilationStateInfo.state === CompilationState.Succeeded || + this.currentCompilationStateInfo.state === CompilationState.Failed + ) { + return this.currentCompilationStateInfo.operatorInputSchemaMap; + } + return {}; + } + public getWorkflowCompilationErrors(): Readonly> { if ( this.currentCompilationStateInfo.state === CompilationState.Succeeded || diff --git a/core/gui/src/app/workspace/service/suggestion-action/suggestion-action.service.ts b/core/gui/src/app/workspace/service/suggestion-action/suggestion-action.service.ts new file mode 100644 index 00000000000..31f083b71f9 --- /dev/null +++ b/core/gui/src/app/workspace/service/suggestion-action/suggestion-action.service.ts @@ -0,0 +1,243 @@ +import { Injectable, NgZone } from "@angular/core"; +import { BehaviorSubject, Observable } from "rxjs"; +import { Workflow } from "../../../common/type/workflow"; +import { WorkflowSuggestion } from "../../types/workflow-suggestion.interface"; +import { WorkflowActionService } from "../workflow-graph/model/workflow-action.service"; +import { WorkflowUtilService } from "../workflow-graph/util/workflow-util.service"; +import { WorkflowPersistService } from "../../../common/service/workflow-persist/workflow-persist.service"; +import { NzMessageService } from "ng-zorro-antd/message"; +import { cloneDeep } from "lodash"; +import { OperatorPredicate } from "../../types/workflow-common.interface"; +import { UntilDestroy, untilDestroyed } from "@ngneat/until-destroy"; +import { WorkflowSuggestionService } from "../workflow-suggestion/workflow-suggestion.service"; // For managing subscriptions in the service if needed + +// @UntilDestroy() // Add if service has long-lived subscriptions to self-manage +@Injectable({ + providedIn: "root", +}) +export class SuggestionActionService { + private activePreviewIdSubject = new BehaviorSubject(null); + public readonly activePreviewId$: Observable = this.activePreviewIdSubject.asObservable(); + + private isInPreviewModeSubject = new BehaviorSubject(false); + public readonly isInPreviewMode$: Observable = this.isInPreviewModeSubject.asObservable(); + + private workflowBeforePreview: Workflow | null = null; + private viewStateBeforeRestore: { zoom: number; tx: number; ty: number } | null = null; + + constructor( + private workflowActionService: WorkflowActionService, + private workflowUtilService: WorkflowUtilService, + private workflowPersistService: WorkflowPersistService, + private messageService: NzMessageService, + private workflowSuggestionService: WorkflowSuggestionService, + private ngZone: NgZone + ) {} + + public getActivePreviewId(): string | null { + return this.activePreviewIdSubject.getValue(); + } + + public isInPreviewMode(): boolean { + return this.isInPreviewModeSubject.getValue(); + } + + private saveWorkflowState(): void { + this.workflowBeforePreview = cloneDeep(this.workflowActionService.getWorkflow()); + const wrapper = this.workflowActionService.getJointGraphWrapper(); + try { + const paper = wrapper.getMainJointPaper(); + const translate = paper.translate(); + this.viewStateBeforeRestore = { zoom: wrapper.getZoomRatio(), tx: translate.tx, ty: translate.ty }; + } catch { + this.viewStateBeforeRestore = null; + } + } + + private restoreWorkflowState(): void { + const snapshot = this.workflowBeforePreview; + this.workflowBeforePreview = null; + + if (snapshot) { + this.workflowActionService.reloadWorkflow(cloneDeep(snapshot)); // Use a deep copy to prevent modifications + } + + if (this.viewStateBeforeRestore) { + // It's better to run this after the graph has been reloaded and rendered. + // Using a timeout ensures that the DOM and JointJS have settled. + setTimeout(() => { + this.ngZone.run(() => { + // Ensure running within Angular's zone if it involves UI updates triggered by JointJS + const wrapper = this.workflowActionService.getJointGraphWrapper(); + const paper = wrapper.getMainJointPaper(); // Get a fresh reference + if (this.viewStateBeforeRestore) { + // Check again as it might be cleared by another async operation + wrapper.setZoomProperty(this.viewStateBeforeRestore.zoom); + paper.translate(this.viewStateBeforeRestore.tx, this.viewStateBeforeRestore.ty); + this.viewStateBeforeRestore = null; + } + }); + }, 0); + } + } + + private applyWorkflowSuggestionChanges( + suggestion: WorkflowSuggestion, + options: { preview: boolean } + ): Map { + const texeraGraph = this.workflowActionService.getTexeraGraph(); + const jointGraph = this.workflowActionService.getJointGraph(); + const jointGraphWrapper = this.workflowActionService.getJointGraphWrapper(); + const operatorIDMap = new Map(); // Maps original suggested ID to newly created ID + const operatorsAndPositions: { op: OperatorPredicate; pos: { x: number; y: number } }[] = []; + + // Handle operators to add or modify properties + suggestion.changes.operatorsToAdd.forEach((opDetails, index) => { + const isExisting = texeraGraph.hasOperator(opDetails.operatorID); + + if (isExisting) { + // Operator exists, so we're modifying its properties + this.workflowActionService.setOperatorProperty(opDetails.operatorID, { ...opDetails.operatorProperties }); + if (options.preview) { + const cell = jointGraph.getCell(opDetails.operatorID); + if (cell) { + cell.attr({ rect: { fill: "rgba(255, 255, 204, 0.6)", stroke: "#1890ff", "stroke-width": 2 } }); + } + } + operatorIDMap.set(opDetails.operatorID, opDetails.operatorID); // Map to itself as it exists + return; + } + + // Operator does not exist, create and add it + const newOp = this.workflowUtilService.getNewOperatorPredicate(opDetails.operatorType); + Object.assign(newOp.operatorProperties, opDetails.operatorProperties); + // Important: Use the newOp.operatorID for mapping and further operations + operatorIDMap.set(opDetails.operatorID, newOp.operatorID); + + let pos = { x: 100, y: 100 + index * 100 }; // Default position + const anchorLink = suggestion.changes.linksToAdd.find(l => l.target.operatorID === opDetails.operatorID); + if (anchorLink) { + const sourceOpInGraphID = texeraGraph.hasOperator(anchorLink.source.operatorID) + ? anchorLink.source.operatorID + : operatorIDMap.get(anchorLink.source.operatorID); + + if (sourceOpInGraphID && texeraGraph.hasOperator(sourceOpInGraphID)) { + const anchorPos = jointGraphWrapper.getElementPosition(sourceOpInGraphID); + pos = { x: anchorPos.x + 200, y: anchorPos.y }; + } + } + operatorsAndPositions.push({ op: newOp, pos }); // Use newOp here + }); + + // Apply additions and deletions + if (!options.preview) { + // For permanent application, delete operators first + if (suggestion.changes.operatorsToDelete.length > 0) { + this.workflowActionService.deleteOperatorsAndLinks(suggestion.changes.operatorsToDelete); // Assuming linksToDelete is empty or handled separately + } + // Then add new/modified ones + this.workflowActionService.addOperatorsAndLinks(operatorsAndPositions, []); // Assuming linksToAdd handled next + } else { + // For preview, just add with visual styling + operatorsAndPositions.forEach(({ op, pos }) => { + this.workflowActionService.addOperator(op, pos); + const cell = jointGraph.getCell(op.operatorID); // Use the actual ID of the added operator + if (cell) { + cell.attr({ ".": { opacity: 0.6 }, rect: { stroke: "#1890ff", "stroke-width": 2 } }); + } + }); + } + + // Handle links to add + suggestion.changes.linksToAdd.forEach(linkDetails => { + // IMPORTANT: Use the operatorIDMap to get the *actual* IDs in the graph + const sourceIDInGraph = operatorIDMap.get(linkDetails.source.operatorID) ?? linkDetails.source.operatorID; + const targetIDInGraph = operatorIDMap.get(linkDetails.target.operatorID) ?? linkDetails.target.operatorID; + + if (texeraGraph.hasOperator(sourceIDInGraph) && texeraGraph.hasOperator(targetIDInGraph)) { + const linkObject = { + linkID: options.preview + ? `link-preview-${Date.now()}-${Math.random().toString(36).substring(2, 7)}` + : `link-${Date.now()}-${Math.random().toString(36).substring(2, 7)}`, // Ensure unique ID + source: { operatorID: sourceIDInGraph, portID: linkDetails.source.portID }, + target: { operatorID: targetIDInGraph, portID: linkDetails.target.portID }, + }; + this.workflowActionService.addLink(linkObject); + + if (options.preview) { + const cell = jointGraph.getCell(linkObject.linkID); + if (cell) { + cell.attr({ + ".connection": { opacity: 0.6, stroke: "#1890ff" }, + ".marker-target": { opacity: 0.6, fill: "#1890ff" }, + }); + } + } + } else { + console.warn( + "Could not add link, source or target operator not found in graph or ID map:", + linkDetails, + "Mapped IDs:", + sourceIDInGraph, + targetIDInGraph + ); + } + }); + return operatorIDMap; + } + + public togglePreview(suggestion: WorkflowSuggestion): void { + const currentPreviewId = this.activePreviewIdSubject.getValue(); + + if (currentPreviewId === suggestion.suggestionID) { + this.clearPreviewAndRestoreWorkflow(); + } else { + if (currentPreviewId) { + this.clearPreviewAndRestoreWorkflow(); // Clear existing preview first + } + this.saveWorkflowState(); + this.activePreviewIdSubject.next(suggestion.suggestionID); + this.isInPreviewModeSubject.next(true); + this.workflowSuggestionService.setPreviewActive(true); // Notify other services + + this.ngZone.runOutsideAngular(() => { + setTimeout(() => { + // Defer to allow UI updates + this.ngZone.run(() => { + // Ensure graph operations run in Angular zone if they trigger changes + this.applyWorkflowSuggestionChanges(suggestion, { preview: true }); + this.messageService.info("Suggestion preview active. Some functionalities might be limited."); + }); + }); + }); + } + } + + public clearPreviewAndRestoreWorkflow(): void { + if (!this.isInPreviewModeSubject.getValue()) return; + + this.restoreWorkflowState(); // Restore graph first + this.activePreviewIdSubject.next(null); + this.isInPreviewModeSubject.next(false); + this.workflowSuggestionService.setPreviewActive(false); // Notify other services + this.messageService.info("Suggestion preview cancelled."); + } + + public applySuggestion(suggestion: WorkflowSuggestion): void { + if (this.isInPreviewModeSubject.getValue()) { + this.clearPreviewAndRestoreWorkflow(); // Clear preview before applying + } + try { + this.applyWorkflowSuggestionChanges(suggestion, { preview: false }); + const workflow = this.workflowActionService.getWorkflow(); + // No need for untilDestroyed(this) if this service is root-provided and lives for app duration + this.workflowPersistService.persistWorkflow(workflow).subscribe(() => { + this.messageService.success("Suggestion applied and workflow saved!"); + this.workflowSuggestionService.removeSuggestionById(suggestion.suggestionID); + }); + } catch (error) { + console.error("Error applying suggestion:", error); + this.messageService.error("Failed to apply the suggestion."); + } + } +} diff --git a/core/gui/src/app/workspace/service/workflow-status/workflow-status.service.ts b/core/gui/src/app/workspace/service/workflow-status/workflow-status.service.ts index 8e483a57d39..7d1f283578e 100644 --- a/core/gui/src/app/workspace/service/workflow-status/workflow-status.service.ts +++ b/core/gui/src/app/workspace/service/workflow-status/workflow-status.service.ts @@ -18,9 +18,10 @@ */ import { Injectable } from "@angular/core"; -import { Observable, Subject } from "rxjs"; +import { BehaviorSubject, Observable, Subject } from "rxjs"; import { OperatorState, OperatorStatistics } from "../../types/execute-workflow.interface"; import { WorkflowWebsocketService } from "../workflow-websocket/workflow-websocket.service"; +import { TableProfile } from "../../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; @Injectable({ providedIn: "root", @@ -30,17 +31,30 @@ export class WorkflowStatusService { private statusSubject = new Subject>(); private currentStatus: Record = {}; + private tableProfileSubject = new BehaviorSubject>({} as Record); + private currentTableProfiles: Record = {}; + constructor(private workflowWebsocketService: WorkflowWebsocketService) { - this.getStatusUpdateStream().subscribe(event => (this.currentStatus = event)); + this.getStatusUpdateStream().subscribe(event => { + this.currentStatus = event; + }); + this.getTableProfilesUpdateStream().subscribe(event => { + this.currentTableProfiles = event; + }); this.workflowWebsocketService.websocketEvent().subscribe(event => { if (event.type !== "OperatorStatisticsUpdateEvent") { return; } this.statusSubject.next(event.operatorStatistics); + this.tableProfileSubject.next(event.operatorResultTableProfiles); }); } + public getTableProfilesUpdateStream(): Observable> { + return this.tableProfileSubject.asObservable(); + } + public getStatusUpdateStream(): Observable> { return this.statusSubject.asObservable(); } @@ -49,6 +63,10 @@ export class WorkflowStatusService { return this.currentStatus; } + public getCurrentTableProfiles(): Record { + return this.currentTableProfiles; + } + public resetStatus(): void { const initStatus: Record = Object.keys(this.currentStatus).reduce( (accumulator, operatorId) => { @@ -62,10 +80,14 @@ export class WorkflowStatusService { {} as Record ); this.statusSubject.next(initStatus); + this.tableProfileSubject.next({}); } public clearStatus(): void { this.currentStatus = {}; this.statusSubject.next({}); + + this.currentTableProfiles = {}; + this.tableProfileSubject.next({}); } } diff --git a/core/gui/src/app/workspace/service/workflow-suggestion/workflow-suggestion.service.ts b/core/gui/src/app/workspace/service/workflow-suggestion/workflow-suggestion.service.ts new file mode 100644 index 00000000000..8cf7ec1421f --- /dev/null +++ b/core/gui/src/app/workspace/service/workflow-suggestion/workflow-suggestion.service.ts @@ -0,0 +1,300 @@ +import { HttpClient, HttpHeaders } from "@angular/common/http"; +import { Injectable, OnDestroy } from "@angular/core"; +import { Observable, of, BehaviorSubject, pipe, timer, delay } from "rxjs"; +import { catchError, map, tap, finalize } from "rxjs/operators"; +import { AppSettings } from "../../../common/app-setting"; +import { Workflow } from "../../../common/type/workflow"; +import { ExecutionStateInfo } from "../../types/execute-workflow.interface"; +import { WorkflowDataCleaningSuggestionList, WorkflowSuggestionList } from "../../types/workflow-suggestion.interface"; +import { v4 as uuid } from "uuid"; +import { CompilationStateInfo, SchemaAttribute } from "../../types/workflow-compiling.interface"; +import { TableProfile } from "../../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; + +// endpoint for workflow suggestions +export const WORKFLOW_SUGGESTION_ENDPOINT = "workflow-suggestion"; +// new endpoint for data cleaning suggestions +export const DATA_CLEANING_SUGGESTION_ENDPOINT = "data-cleaning-suggestion"; + +// Define the request interface if not already globally available +export interface TableProfileSuggestionRequest { + focusingOperatorID: string; + tableSchema: ReadonlyArray; + tableProfile: TableProfile; + targetColumnName: string; +} + +/** + * WorkflowSuggestionService is responsible for communicating with the backend suggestion service. + * It gathers the necessary data (workflow, compilation state, and result data) and sends it to + * the backend to generate workflow suggestions. + */ +@Injectable({ + providedIn: "root", +}) +export class WorkflowSuggestionService implements OnDestroy { + // Stream that indicates whether a preview is currently active - initialized to false + private previewActiveStream = new BehaviorSubject(false); + // Stream that always holds the latest list of suggestions returned from the backend. + private suggestionsListSubject = new BehaviorSubject({ suggestions: [] }); + // Stream indicating whether a request is in flight so that components can show loading states. + private suggestionsLoadingSubject = new BehaviorSubject(false); + // Flag to ignore workflow changes during preview activation/deactivation + private mock = false; // Set to true to enable mock mode + + constructor(private httpClient: HttpClient) { + // Ensure preview is false on initial load + this.resetPreviewState(); + } + + ngOnDestroy(): void { + // Make sure preview state is reset when service is destroyed + this.resetPreviewState(); + } + + /** + * Observable stream of the latest workflow suggestion list. Components that want to + * reactively display suggestions (for example the SuggestionFrameComponent) should subscribe + * to this stream instead of (or in addition to) directly calling getSuggestions(). Whenever + * getSuggestions() successfully receives a response from the backend this stream will emit + * the same suggestion list so that every interested component stays in sync. + */ + public getSuggestionsListStream(): Observable { + return this.suggestionsListSubject.asObservable(); + } + + /** + * Observable stream indicating whether suggestions are currently being fetched. + */ + public getLoadingStream(): Observable { + return this.suggestionsLoadingSubject.asObservable(); + } + + /** + * Requests workflow suggestions from the backend service. + * This method gathers the current workflow state, compilation information, + * and result data, then sends it to the backend to generate suggestions. + * + * @returns Observable of workflow suggestions + */ + public getSuggestions( + workflow: Workflow, + compilationState: CompilationStateInfo, + executionState: ExecutionStateInfo, + intention: string, + focusingOperatorIDs: readonly string[], + operatorIDToTableSchemaMap: Record> + ): Observable { + // indicate loading started + this.suggestionsLoadingSubject.next(true); + + // Helper to mark request finished + const done = () => this.suggestionsLoadingSubject.next(false); + + // Skip if preview is active + if (this.previewActiveStream.getValue()) { + return of({ suggestions: [] }).pipe( + tap(list => this.suggestionsListSubject.next(list)), + finalize(done) + ); + } + + if (this.mock) { + return of(this.MOCK_SUGGESTIONS).pipe( + tap(list => this.suggestionsListSubject.next(list)), + finalize(done) + ); + } + + return this.httpClient + .post(`${AppSettings.getApiEndpoint()}/${WORKFLOW_SUGGESTION_ENDPOINT}`, { + workflow: JSON.stringify(workflow), + compilationState: compilationState, + executionState: executionState, + intention: intention, + focusingOperatorIDs: focusingOperatorIDs, + operatorIDToTableSchemaMap: operatorIDToTableSchemaMap, + }) + .pipe( + map(suggestionList => { + suggestionList.suggestions.forEach(suggestion => { + suggestion.suggestionID = `suggestion-${uuid()}`; + }); + return suggestionList; + }), + // Publish the suggestion list so that other components (e.g. the suggestion frame) + // can react to the new data even if they did not initiate the request. + tap(suggestionList => this.suggestionsListSubject.next(suggestionList)), + catchError((error: unknown) => { + console.error("Error getting workflow suggestions:", error); + // publish empty list on error + this.suggestionsListSubject.next({ suggestions: [] }); + return of({ suggestions: [] }); + }), + finalize(done) + ); + } + + /** + * Get an observable stream indicating whether a preview is active. + * Components can subscribe to this to know when to skip certain operations. + * + * @returns Observable boolean indicating if a preview is active + */ + public getPreviewActiveStream(): Observable { + return this.previewActiveStream.asObservable(); + } + + /** + * Set whether a preview is currently active. + * This will notify all subscribers to the preview active stream. + * + * @param isActive Whether a preview is currently active + */ + public setPreviewActive(isActive: boolean): void { + console.log(`WorkflowSuggestionService: Setting preview active to ${isActive}`); + + // Set the preview state + this.previewActiveStream.next(isActive); + } + + /** + * Reset the preview state to false. + * This should be called when navigating away from the workspace or + * when the component is destroyed. + */ + public resetPreviewState(): void { + this.setPreviewActive(false); + } + + /** Remove a suggestion with given ID from current list and notify subscribers. */ + public removeSuggestionById(id: string): void { + const current = this.suggestionsListSubject.getValue(); + const newList = { suggestions: current.suggestions.filter(s => s.suggestionID !== id) }; + this.suggestionsListSubject.next(newList); + } + + /** + * Requests data cleaning suggestions from the backend service based on table profile and target column. + * @param focusingOperatorID + * @param tableSchema + * @param tableProfile The complete table profile. + * @param targetColumnName The name of the column for which to get suggestions. + * @returns Observable of SuggestionList + */ + public getDataCleaningSuggestions( + focusingOperatorID: string, + tableSchema: ReadonlyArray, + tableProfile: TableProfile, + targetColumnName: string + ): Observable { + const requestPayload: TableProfileSuggestionRequest = { + focusingOperatorID: focusingOperatorID, + tableSchema: tableSchema, + tableProfile: tableProfile, + targetColumnName: targetColumnName, + }; + + return this.httpClient + .post( + `${AppSettings.getApiEndpoint()}/${DATA_CLEANING_SUGGESTION_ENDPOINT}`, + requestPayload + ) + .pipe( + map(suggestionList => { + // Ensure suggestionIDs are unique if backend doesn't guarantee it + suggestionList.suggestions.forEach(suggestion => { + if (!suggestion.suggestionID) { + suggestion.suggestionID = `dcsuggestion-${uuid()}`; + } + }); + return suggestionList; + }), + catchError((error: unknown) => { + console.error("Error getting data cleaning suggestions:", error); + return of({ suggestions: [] } as WorkflowDataCleaningSuggestionList); // Return empty list on error + }) + ); + } + + MOCK_SUGGESTIONS: WorkflowSuggestionList = { + suggestions: [ + { + suggestionID: "suggestion-0", + suggestion: "Add aggregation to compute summary statistics for tweet data.", + suggestionType: "improve", + changes: { + operatorsToAdd: [ + { + operatorType: "Aggregate", + operatorID: "Aggregate-operator-new-1", + operatorProperties: { + aggregations: [ + { + aggFunction: "sum", + attribute: "favorite_count", + "result attribute": "total_favorite_count", + }, + { + aggFunction: "average", + attribute: "retweet_count", + "result attribute": "average_retweet_count", + }, + ], + groupByKeys: ["create_at_month"], + }, + customDisplayName: "Aggregate Tweet Data", + }, + ], + linksToAdd: [ + { + linkID: "link-8af0915c-6ccc-439a-bf05-1b39cefdbbfd", + source: { + operatorID: "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2", + portID: "output-0", + }, + target: { + operatorID: "Aggregate-operator-new-1", + portID: "input-0", + }, + }, + ], + operatorsToDelete: [], + }, + }, + { + suggestionID: "suggestion-1", + suggestion: "Enhance the workflow with sentiment analysis of tweets.", + suggestionType: "improve", + changes: { + operatorsToAdd: [ + { + operatorType: "HuggingFaceSentimentAnalysis", + operatorID: "HuggingFaceSentimentAnalysis-5226", + operatorProperties: { + attribute: "text", + "Positive result attribute": "positive_sentiment", + "Neutral result attribute": "neutral_sentiment", + "Negative result attribute": "negative_sentiment", + }, + customDisplayName: "Sentiment Analysis", + }, + ], + linksToAdd: [ + { + linkID: "link-e20f28c0-2da3-406c-be8a-df2a41ff22e3", + source: { + operatorID: "PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c", + portID: "output-0", + }, + target: { + operatorID: "HuggingFaceSentimentAnalysis-5226", + portID: "input-0", + }, + }, + ], + operatorsToDelete: [], + }, + }, + ], + }; +} diff --git a/core/gui/src/app/workspace/types/execute-workflow.interface.ts b/core/gui/src/app/workspace/types/execute-workflow.interface.ts index 3ede29ab9a3..7032790d62d 100644 --- a/core/gui/src/app/workspace/types/execute-workflow.interface.ts +++ b/core/gui/src/app/workspace/types/execute-workflow.interface.ts @@ -24,6 +24,7 @@ */ import { WorkflowFatalError, OperatorCurrentTuples } from "./workflow-websocket.interface"; +import { TableProfile } from "../../common/type/proto/edu/uci/ics/amber/engine/architecture/worker/tableprofile"; export interface PortIdentity extends Readonly<{ id: number; @@ -87,6 +88,7 @@ export interface OperatorStatistics export interface OperatorStatsUpdate extends Readonly<{ operatorStatistics: Record; + operatorResultTableProfiles: Record; }> {} export type PaginationMode = { type: "PaginationMode" }; diff --git a/core/gui/src/app/workspace/types/workflow-suggestion.interface.ts b/core/gui/src/app/workspace/types/workflow-suggestion.interface.ts new file mode 100644 index 00000000000..bbab2ad1a84 --- /dev/null +++ b/core/gui/src/app/workspace/types/workflow-suggestion.interface.ts @@ -0,0 +1,38 @@ +import { OperatorLink, OperatorPredicate } from "./workflow-common.interface"; + +/** + * OperatorSuggestion is a lightweight subset of OperatorPredicate used in suggestions. + */ +export type OperatorSuggestion = Pick< + OperatorPredicate, + "operatorID" | "operatorType" | "customDisplayName" | "operatorProperties" +>; + +export interface Changes { + operatorsToAdd: OperatorSuggestion[]; + linksToAdd: OperatorLink[]; + operatorsToDelete: string[]; +} + +export type SuggestionType = "fix" | "improve"; + +export interface WorkflowSuggestion { + suggestionID: string; + suggestion: string; + suggestionType: SuggestionType; + changes: Changes; +} + +export interface WorkflowSuggestionList { + suggestions: WorkflowSuggestion[]; +} + +export interface WorkflowDataCleaningSuggestion { + suggestionID: string; + suggestion: string; + details: string; +} + +export interface WorkflowDataCleaningSuggestionList { + suggestions: WorkflowDataCleaningSuggestion[]; +} diff --git a/core/gui/src/assets/operator_images/IntelligentCSVReader.png b/core/gui/src/assets/operator_images/IntelligentCSVReader.png new file mode 100644 index 00000000000..d0dfd1a8f9a Binary files /dev/null and b/core/gui/src/assets/operator_images/IntelligentCSVReader.png differ diff --git a/core/gui/yarn.lock b/core/gui/yarn.lock index eefc75f7e67..379c74e9108 100644 --- a/core/gui/yarn.lock +++ b/core/gui/yarn.lock @@ -4915,6 +4915,38 @@ __metadata: languageName: node linkType: hard +"@swimlane/ngx-charts@npm:22.0.0": + version: 22.0.0 + resolution: "@swimlane/ngx-charts@npm:22.0.0" + dependencies: + d3-array: "npm:^3.2.0" + d3-brush: "npm:^3.0.0" + d3-color: "npm:^3.1.0" + d3-ease: "npm:^3.0.1" + d3-format: "npm:^3.1.0" + d3-hierarchy: "npm:^3.1.2" + d3-interpolate: "npm:^3.0.1" + d3-sankey: "npm:^0.12.3" + d3-scale: "npm:^4.0.2" + d3-selection: "npm:^3.0.0" + d3-shape: "npm:^3.2.0" + d3-time-format: "npm:^4.1.0" + d3-transition: "npm:^3.0.1" + gradient-path: "npm:^2.3.0" + tslib: "npm:^2.3.1" + peerDependencies: + "@angular/animations": 17.x || 18.x || 19.x + "@angular/cdk": 17.x || 18.x || 19.x + "@angular/common": 17.x || 18.x || 19.x + "@angular/core": 17.x || 18.x || 19.x + "@angular/forms": 17.x || 18.x || 19.x + "@angular/platform-browser": 17.x || 18.x || 19.x + "@angular/platform-browser-dynamic": 17.x || 18.x || 19.x + rxjs: 7.x + checksum: 10c0/476ed95427f00e65d42625e3ccdecb12549ac735e24029572277af726489f5bb8b4a1890fcca12e6b7e9bf01118c0f2d4531b040a7bf5c51b380e4a8ec4a912e + languageName: node + linkType: hard + "@tootallnate/once@npm:1": version: 1.1.2 resolution: "@tootallnate/once@npm:1.1.2" @@ -5434,6 +5466,13 @@ __metadata: languageName: node linkType: hard +"@types/tinycolor2@npm:^1.4.0": + version: 1.4.6 + resolution: "@types/tinycolor2@npm:1.4.6" + checksum: 10c0/922020c3326460e9d8502c8a98f80db69f06fd14e07fe5a48e8ffe66175762298a9bd51263f2a0c9a40632886a74975a3ff79396defcdbeac0dc176e3e5056e8 + languageName: node + linkType: hard + "@types/underscore@npm:*": version: 1.13.0 resolution: "@types/underscore@npm:1.13.0" @@ -8243,6 +8282,15 @@ __metadata: languageName: node linkType: hard +"d3-array@npm:1 - 2": + version: 2.12.1 + resolution: "d3-array@npm:2.12.1" + dependencies: + internmap: "npm:^1.0.0" + checksum: 10c0/7eca10427a9f113a4ca6a0f7301127cab26043fd5e362631ef5a0edd1c4b2dd70c56ed317566700c31e4a6d88b55f3951aaba192291817f243b730cb2352882e + languageName: node + linkType: hard + "d3-array@npm:2 - 3, d3-array@npm:2.10.0 - 3, d3-array@npm:2.5.0 - 3, d3-array@npm:3, d3-array@npm:^3.2.0": version: 3.2.4 resolution: "d3-array@npm:3.2.4" @@ -8259,7 +8307,7 @@ __metadata: languageName: node linkType: hard -"d3-brush@npm:3": +"d3-brush@npm:3, d3-brush@npm:^3.0.0": version: 3.0.0 resolution: "d3-brush@npm:3.0.0" dependencies: @@ -8281,7 +8329,7 @@ __metadata: languageName: node linkType: hard -"d3-color@npm:1 - 3, d3-color@npm:3": +"d3-color@npm:1 - 3, d3-color@npm:3, d3-color@npm:^3.1.0": version: 3.1.0 resolution: "d3-color@npm:3.1.0" checksum: 10c0/a4e20e1115fa696fce041fbe13fbc80dc4c19150fa72027a7c128ade980bc0eeeba4bcf28c9e21f0bce0e0dbfe7ca5869ef67746541dcfda053e4802ad19783c @@ -8344,7 +8392,7 @@ __metadata: languageName: node linkType: hard -"d3-ease@npm:1 - 3, d3-ease@npm:3": +"d3-ease@npm:1 - 3, d3-ease@npm:3, d3-ease@npm:^3.0.1": version: 3.0.1 resolution: "d3-ease@npm:3.0.1" checksum: 10c0/fec8ef826c0cc35cda3092c6841e07672868b1839fcaf556e19266a3a37e6bc7977d8298c0fcb9885e7799bfdcef7db1baaba9cd4dcf4bc5e952cf78574a88b0 @@ -8371,7 +8419,7 @@ __metadata: languageName: node linkType: hard -"d3-format@npm:1 - 3, d3-format@npm:3": +"d3-format@npm:1 - 3, d3-format@npm:3, d3-format@npm:^3.1.0": version: 3.1.0 resolution: "d3-format@npm:3.1.0" checksum: 10c0/049f5c0871ebce9859fc5e2f07f336b3c5bfff52a2540e0bac7e703fce567cd9346f4ad1079dd18d6f1e0eaa0599941c1810898926f10ac21a31fd0a34b4aa75 @@ -8387,14 +8435,14 @@ __metadata: languageName: node linkType: hard -"d3-hierarchy@npm:3": +"d3-hierarchy@npm:3, d3-hierarchy@npm:^3.1.2": version: 3.1.2 resolution: "d3-hierarchy@npm:3.1.2" checksum: 10c0/6dcdb480539644aa7fc0d72dfc7b03f99dfbcdf02714044e8c708577e0d5981deb9d3e99bbbb2d26422b55bcc342ac89a0fa2ea6c9d7302e2fc0951dd96f89cf languageName: node linkType: hard -"d3-interpolate@npm:1 - 3, d3-interpolate@npm:1.2.0 - 3, d3-interpolate@npm:3": +"d3-interpolate@npm:1 - 3, d3-interpolate@npm:1.2.0 - 3, d3-interpolate@npm:3, d3-interpolate@npm:^3.0.1": version: 3.0.1 resolution: "d3-interpolate@npm:3.0.1" dependencies: @@ -8403,6 +8451,13 @@ __metadata: languageName: node linkType: hard +"d3-path@npm:1": + version: 1.0.9 + resolution: "d3-path@npm:1.0.9" + checksum: 10c0/e35e84df5abc18091f585725b8235e1fa97efc287571585427d3a3597301e6c506dea56b11dfb3c06ca5858b3eb7f02c1bf4f6a716aa9eade01c41b92d497eb5 + languageName: node + linkType: hard + "d3-path@npm:1 - 3, d3-path@npm:3, d3-path@npm:^3.1.0": version: 3.1.0 resolution: "d3-path@npm:3.1.0" @@ -8431,6 +8486,16 @@ __metadata: languageName: node linkType: hard +"d3-sankey@npm:^0.12.3": + version: 0.12.3 + resolution: "d3-sankey@npm:0.12.3" + dependencies: + d3-array: "npm:1 - 2" + d3-shape: "npm:^1.2.0" + checksum: 10c0/261debb01a13269f6fc53b9ebaef174a015d5ad646242c23995bf514498829ab8b8f920a7873724a7494288b46bea3ce7ebc5a920b745bc8ae4caa5885cf5204 + languageName: node + linkType: hard + "d3-scale-chromatic@npm:3": version: 3.1.0 resolution: "d3-scale-chromatic@npm:3.1.0" @@ -8441,7 +8506,7 @@ __metadata: languageName: node linkType: hard -"d3-scale@npm:4": +"d3-scale@npm:4, d3-scale@npm:^4.0.2": version: 4.0.2 resolution: "d3-scale@npm:4.0.2" dependencies: @@ -8454,14 +8519,14 @@ __metadata: languageName: node linkType: hard -"d3-selection@npm:2 - 3, d3-selection@npm:3": +"d3-selection@npm:2 - 3, d3-selection@npm:3, d3-selection@npm:^3.0.0": version: 3.0.0 resolution: "d3-selection@npm:3.0.0" checksum: 10c0/e59096bbe8f0cb0daa1001d9bdd6dbc93a688019abc97d1d8b37f85cd3c286a6875b22adea0931b0c88410d025563e1643019161a883c516acf50c190a11b56b languageName: node linkType: hard -"d3-shape@npm:3": +"d3-shape@npm:3, d3-shape@npm:^3.2.0": version: 3.2.0 resolution: "d3-shape@npm:3.2.0" dependencies: @@ -8470,7 +8535,16 @@ __metadata: languageName: node linkType: hard -"d3-time-format@npm:2 - 4, d3-time-format@npm:4": +"d3-shape@npm:^1.2.0": + version: 1.3.7 + resolution: "d3-shape@npm:1.3.7" + dependencies: + d3-path: "npm:1" + checksum: 10c0/548057ce59959815decb449f15632b08e2a1bdce208f9a37b5f98ec7629dda986c2356bc7582308405ce68aedae7d47b324df41507404df42afaf352907577ae + languageName: node + linkType: hard + +"d3-time-format@npm:2 - 4, d3-time-format@npm:4, d3-time-format@npm:^4.1.0": version: 4.1.0 resolution: "d3-time-format@npm:4.1.0" dependencies: @@ -8495,7 +8569,7 @@ __metadata: languageName: node linkType: hard -"d3-transition@npm:2 - 3, d3-transition@npm:3": +"d3-transition@npm:2 - 3, d3-transition@npm:3, d3-transition@npm:^3.0.1": version: 3.0.1 resolution: "d3-transition@npm:3.0.1" dependencies: @@ -11054,6 +11128,15 @@ __metadata: languageName: node linkType: hard +"gradient-path@npm:^2.3.0": + version: 2.3.0 + resolution: "gradient-path@npm:2.3.0" + dependencies: + tinygradient: "npm:^1.0.0" + checksum: 10c0/5008d2c89b1985a41e5ac8932285b934a9ffed1a8fd26b006be8770c10be2fc00f3b8038b2373c387595307ad659bf0f396365756ec781301ef9348d0f444b8e + languageName: node + linkType: hard + "graphemer@npm:^1.4.0": version: 1.4.0 resolution: "graphemer@npm:1.4.0" @@ -11117,6 +11200,7 @@ __metadata: "@nrwl/nx-cloud": "npm:19.1.0" "@nx/angular": "npm:20.0.3" "@stoplight/json-ref-resolver": "npm:3.1.5" + "@swimlane/ngx-charts": "npm:22.0.0" "@types/backbone": "npm:1.4.15" "@types/content-disposition": "npm:0" "@types/dagre": "npm:0.7.47" @@ -11833,6 +11917,13 @@ __metadata: languageName: node linkType: hard +"internmap@npm:^1.0.0": + version: 1.0.1 + resolution: "internmap@npm:1.0.1" + checksum: 10c0/60942be815ca19da643b6d4f23bd0bf4e8c97abbd080fb963fe67583b60bdfb3530448ad4486bae40810e92317bded9995cc31411218acc750d72cd4e8646eee + languageName: node + linkType: hard + "ip-address@npm:^9.0.5": version: 9.0.5 resolution: "ip-address@npm:9.0.5" @@ -17654,6 +17745,23 @@ __metadata: languageName: node linkType: hard +"tinycolor2@npm:^1.0.0": + version: 1.6.0 + resolution: "tinycolor2@npm:1.6.0" + checksum: 10c0/9aa79a36ba2c2a87cb221453465cabacd04b9e35f9694373e846fdc78b1c768110f81e581ea41440106c0f24d9a023891d0887e8075885e790ac40eb0e74a5c1 + languageName: node + linkType: hard + +"tinygradient@npm:^1.0.0": + version: 1.1.5 + resolution: "tinygradient@npm:1.1.5" + dependencies: + "@types/tinycolor2": "npm:^1.4.0" + tinycolor2: "npm:^1.0.0" + checksum: 10c0/00503406e1a49822e58e90ddbda7e9db332b6a237fab5f413be9644eb29ca14f6a30e7961fc61328407fa6159437908c11eeda4f1d56e62586f9a48267d1094d + languageName: node + linkType: hard + "tinyqueue@npm:2.0.3": version: 2.0.3 resolution: "tinyqueue@npm:2.0.3" diff --git a/core/scripts/gui-proto-gen.sh b/core/scripts/gui-proto-gen.sh index e3576a577e3..94c1c7597eb 100755 --- a/core/scripts/gui-proto-gen.sh +++ b/core/scripts/gui-proto-gen.sh @@ -17,15 +17,19 @@ TEXERA_ROOT="$(git rev-parse --show-toplevel)" GUI_DIR="$TEXERA_ROOT/core/gui" -PROTOBUF_DIR="$TEXERA_ROOT/core/workflow-core/src/main/protobuf" +PROTOBUF_AMBER_DIR="$TEXERA_ROOT/core/amber/src/main/protobuf" +PROTOBUF_WORKFLOW_CORE_DIR="$TEXERA_ROOT/core/workflow-core/src/main/protobuf" GUI_PROTO_DIR="$GUI_DIR/src/app/common/type" -WORKFLOW_PROTO=$(find "$PROTOBUF_DIR" -iname "workflow.proto") -VIRTUALIDENTITY_PROTO=$(find "$PROTOBUF_DIR" -iname "virtualidentity.proto") +WORKFLOW_PROTO=$(find "$PROTOBUF_WORKFLOW_CORE_DIR" -iname "workflow.proto") +VIRTUALIDENTITY_PROTO=$(find "$PROTOBUF_WORKFLOW_CORE_DIR" -iname "virtualidentity.proto") +TABLEPROFILE_PROTO=$(find "$PROTOBUF_AMBER_DIR" -iname "tableprofile.proto") + protoc --plugin="$GUI_DIR/node_modules/.bin/protoc-gen-ts_proto" \ --ts_proto_out="$GUI_PROTO_DIR/proto" \ - -I="$PROTOBUF_DIR" \ + -I="$PROTOBUF_AMBER_DIR" \ + -I="$PROTOBUF_WORKFLOW_CORE_DIR" \ "$WORKFLOW_PROTO" \ "$VIRTUALIDENTITY_PROTO" \ - --proto_path="$PROTOBUF_DIR" \ No newline at end of file + "$TABLEPROFILE_PROTO" \ No newline at end of file diff --git a/core/scripts/suggestion-service-proto-gen.sh b/core/scripts/suggestion-service-proto-gen.sh new file mode 100644 index 00000000000..2dc5f692175 --- /dev/null +++ b/core/scripts/suggestion-service-proto-gen.sh @@ -0,0 +1,34 @@ +# 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. + +#!/bin/bash + +TEXERA_ROOT="$(git rev-parse --show-toplevel)" +AMBER_DIR="$TEXERA_ROOT/core/amber" +PROTOBUF_AMBER_DIR="$AMBER_DIR/src/main/protobuf" +CORE_DIR="$TEXERA_ROOT/core/workflow-core" +PROTOBUF_CORE_DIR="$CORE_DIR/src/main/protobuf" + +OUT_DIR="$TEXERA_ROOT/core/suggestion-service/model/proto" + +# target proto +TABLEPROFILE_PROTO=$(find "$PROTOBUF_AMBER_DIR" -iname "tableprofile.proto") + +# generate only that file +protoc --python_betterproto_out="$OUT_DIR" \ + -I="$PROTOBUF_AMBER_DIR" \ + -I="$PROTOBUF_CORE_DIR" \ + "$TABLEPROFILE_PROTO" \ No newline at end of file diff --git a/core/suggestion-service/.gitignore b/core/suggestion-service/.gitignore new file mode 100644 index 00000000000..eb6b8bc6268 --- /dev/null +++ b/core/suggestion-service/.gitignore @@ -0,0 +1,84 @@ +# Python bytecode +__pycache__/ +*.py[cod] +*$py.class + +# Distribution / packaging +dist/ +build/ +*.egg-info/ +*.egg +.eggs/ +.installed.cfg +develop-eggs/ +downloads/ +eggs/ +parts/ +sdist/ +var/ +wheels/ + +# Unit test / coverage reports +.pytest_cache/ +.coverage +htmlcov/ +.tox/ +.nox/ +.hypothesis/ +.pytest_cache/ +coverage.xml +*.cover + +# Virtual environments +venv/ +env/ +ENV/ +.env +.venv +pythonenv* +.python-version + +# Development environments +.idea/ +.vscode/ +*.swp +*.swo +*~ + +# Jupyter Notebook +.ipynb_checkpoints + +# Type checking +.mypy_cache/ +.dmypy.json +dmypy.json +.pyre/ +.pytype/ + +# ML specific +*.model +*.pkl +*.h5 +*.joblib +*.pt +model_checkpoints/ + +# Logs +logs/ +*.log + +# Local development settings +local_settings.py +instance/ + +# Database +*.db +*.sqlite +*.sqlite3 + +# Large media files that might get added accidentally +*.mp4 +*.mov +*.wav +*.mp3 +.env \ No newline at end of file diff --git a/core/suggestion-service/README.md b/core/suggestion-service/README.md new file mode 100644 index 00000000000..8eb6817581a --- /dev/null +++ b/core/suggestion-service/README.md @@ -0,0 +1,46 @@ +## Workflow Suggestion Generator + +### Prompt Generation Layer + +This layer is responsible for processing the given information regarding the workflow, compilation state, execution state and result, into the better natural language description. Once these natural language descriptions are generated, they can be passed to the LLM agent. + +Prompt generation can be divided into 2 major packages, each package provides a single endpoint: +#### 1. Workflow Static Interpretation +- Endpoint name: interpretWorkflow +- Endpoint intput parameters: + - workflow (dict) + - input schema for each operator (dict) + - static error for each operator (dict) + - interpretation method type + 1. RAW: simply use the below template + ``` + Here is the workflow dict: + ${string representation of workflow dict} + + Here is the input schema for each operator: + ${string representation of the input schema dict} + ``` + 2. BY_PATH + - use the `TexeraWorkflow` class to parse the workflow dict and input schema dict + - parse out all the paths from the DAG structure. A path is a sub workflow which is a single line, from source operator to the sink operator. This subworkflow should also carry the schema information extracted from the input schema dict, and the static error information, on each operator + + - use the below template to generate the description from the paths and schemas: + ``` + Here are the existing paths in this workflow and related schemas + + Path1: ${string representation of the path's sub-workflow} + + Path2: ${string representation of the path's sub-workflow} + ... + ``` +- Endpoint output parameters: + - natural language description (a string) + + + +#### 2. Workflow Interpretation with Execution information +TODO + +### LLM Agent Layer + +This layer is responsible for calling language models to get the response (the suggestions). diff --git a/core/suggestion-service/app.py b/core/suggestion-service/app.py new file mode 100644 index 00000000000..f9294580393 --- /dev/null +++ b/core/suggestion-service/app.py @@ -0,0 +1,86 @@ +import json +import os +from dotenv import load_dotenv + +from fastapi import FastAPI, HTTPException +from fastapi.middleware.cors import CORSMiddleware + +from model.llm.suggestion import ( + SuggestionList, + Suggestion, + Changes, + DataCleaningSuggestionList, +) +from model.web.input import SuggestionRequest, TableProfileSuggestionRequest +from model.llm.interpretation import InterpretationMethod +from suggestion_engine.generator import SuggestionGenerator + +# Load environment variables +load_dotenv() + +app = FastAPI(title="Texera Workflow Suggestion Service") + +# Enable CORS +app.add_middleware( + CORSMiddleware, + allow_origins=["*"], # In production, restrict this to your frontend domain + allow_credentials=True, + allow_methods=["*"], + allow_headers=["*"], +) + + +# Initialize the suggestion generator with LLM support +suggestion_generator = SuggestionGenerator() + +# @app.middleware("http") +# async def log_raw_body(request: Request, call_next): +# body = await request.body() +# print("🔥 RAW REQUEST BODY:", body.decode()) +# response = await call_next(request) +# return response + + +@app.get("/") +async def root(): + return {"message": "Texera Workflow Suggestion Service is running"} + + +@app.post("/api/workflow-suggestion", response_model=SuggestionList) +async def generate_suggestions(request: SuggestionRequest): + """ + Generate workflow suggestions based on the current workflow, compilation state, and execution state. + """ + try: + # Generate suggestions using the suggestion engine + suggestions = suggestion_generator.generate_suggestions(request) + + return suggestions + + except json.JSONDecodeError: + raise HTTPException(status_code=400, detail="Invalid workflow JSON format") + except Exception as e: + raise HTTPException( + status_code=500, detail=f"Error generating suggestions: {str(e)}" + ) + + +@app.post("/api/data-cleaning-suggestion", response_model=DataCleaningSuggestionList) +async def generate_table_profile_suggestions(request: TableProfileSuggestionRequest): + """ + Generate suggestions based on table profile and a target column. + """ + try: + suggestions = suggestion_generator.generate_data_cleaning_suggestions(request) + return suggestions + + except Exception as e: + raise HTTPException( + status_code=500, detail=f"Error generating suggestions: {str(e)}" + ) + + +if __name__ == "__main__": + import uvicorn + + uvicorn.run(app, host="0.0.0.0", port=9094) diff --git a/core/suggestion-service/files/instruction_for_data_cleaning_agent.md b/core/suggestion-service/files/instruction_for_data_cleaning_agent.md new file mode 100644 index 00000000000..815b8f0c484 --- /dev/null +++ b/core/suggestion-service/files/instruction_for_data_cleaning_agent.md @@ -0,0 +1,102 @@ +# Identity + +You are an AI assistant that reviews **a single column profile** of an operator's result table in a Texera workflow and recommends the **next data-cleaning step for this column**. +Your purpose is to propose the suggestion in text of guiding users toward improving data quality during the data cleaning stage. + +# Instructions + +You will receive **one JSON-object string** containing the following three fields: + +## focusingOperatorID +The ID of the Operator that produces the table containing the target column. The suggestion details should be explicit using this operatorID, i.e. After the operator with operatorID..., do ... + +## tableSchema +List of column type and column name of this table. + +## columnProfile +An object containing multiple fields about the statistics of the column: + - `column_name` *(string)* — column header / label in the dataset + - `data_type` *(string)* — primitive Python type detected (`int`, `float`, `str`, …) + - `data_label` *(string)* — semantic label assigned by the Labeler component + - `categorical` *(bool)* — `true` if the column contains categorical values + - `order` *(string)* — ordering of values (`ascending`, `descending`, or `random`) + - `samples` *(array\)* — a handful of example entries + - `statistics` *(object)* — **see nested fields below** + - `sample_size` *(int)* — number of rows examined + - `null_count` *(int)* — total null entries + - `null_types` *(array\)* — distinct null representations (`""`, `"NA"`, `null`, …) + - `null_types_index` *(object)* — map: null type → list of row indices + - `data_type_representation` *(object)* — map: data type → share of sample rows + - `min` / `max` *(number)* — extreme values + - `mode` *(mixed)* — most frequent value + - `median` *(number)* — median of numeric entries + - `median_absolute_deviation` *(number)* — MAD around the median + - `sum` *(number)* — sum of numeric values + - `mean` *(number)* — arithmetic mean + - `variance` *(number)* — variance + - `stddev` *(number)* — standard deviation + - `skewness` *(number)* — skewness coefficient + - `kurtosis` *(number)* — kurtosis coefficient + - `num_zeros` *(int)* — count of exact-zero entries + - `num_negatives` *(int)* — count of values \< 0 + - `histogram.bin_counts` *(array\)* — frequencies per bin + - `histogram.bin_edges` *(array\)* — bin thresholds (length = `bin_counts` + 1) + - `quantiles` *(array\)* — values at selected percentiles (25-50-75 %) + - `vocab` *(array\)* — unique characters present (string columns) + - `avg_predictions` *(number)* — mean confidence of label predictions + - `categories` *(array\)* — distinct categories (`categorical = true`) + - `unique_count` *(int)* — number of distinct values + - `unique_ratio` *(number)* — `unique_count / sample_size` + - `categorical_count` *(object)* — map: category → frequency (`categorical = true`) + - `gini_impurity` *(number)* — Gini impurity of the distribution + - `unalikeability` *(number)* — measure of pairwise disagreement + - `precision` *(object)* — digit-precision stats per numeric value + - `times` *(object)* — profiling runtime in ms (`rowStatsMs`, etc.) + - `format` *(array\)* — candidate datetime formats (if applicable) + +* Use these metrics to detect issues such as high null ratios, redundant formatting, outliers, skewed distributions, mixed types, encoding problems, etc. + + +* Generate **1-2** suggestions. Each suggestion **must** have: + - **`suggestion`** – a short, imperative headline (e.g., “Impute missing values with median”). + - **`details`** – 2-3 sentences explaining *why* and the detailed action plan (method / parameter hints). + - In the details, you MUST mention the operator type you want to use and describe the parameters you will set. You can only use the PythonUDFV2. + - PythonUDFV2: performs the customized data cleaning logic. PythonUDFV2 should be used for complex data manipulation. You MUST describe which API to use and the the logics in the python code. + There are 2 APIs to process the data in different units. + 1. Tuple API. + + ```python + class ProcessTupleOperator(UDFOperatorV2): + + def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]: + yield tuple_ + + ``` + Tuple API takes one input tuple from a port at a time. It returns an iterator of optional `TupleLike` instances. A `TupleLike` is any data structure that supports key-value pairs, such as `pytexera.Tuple`, `dict`, `defaultdict`, `NamedTuple`, etc. + Tuple API is useful for implementing functional operations which are applied to tuples one by one, such as map, reduce, and filter. + + 2. Table API. + + ```python + class ProcessTableOperator(UDFTableOperator): + + def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]: + yield table + ``` + Table API consumes a `Table` at a time, which consists of all the whole table from a port. It returns an iterator of optional `TableLike` instances. A `TableLike ` is a collection of `TupleLike`, and currently, we support `pytexera.Table` and `pandas.DataFrame` as a `TableLike` instance. + Table API is useful for implementing blocking operations that will consume the whole column to do operations. + +* **Respond with exactly one JSON object** that conforms to the schema +`DataCleaningSuggestionList`: + +```jsonc +{ + "suggestions": [ + { + "suggestion": "string", + "details": "string" + } + // … 1-2 items total + ] +} +``` diff --git a/core/suggestion-service/files/instruction_for_function_call.md b/core/suggestion-service/files/instruction_for_function_call.md new file mode 100644 index 00000000000..a04a481dcf6 --- /dev/null +++ b/core/suggestion-service/files/instruction_for_function_call.md @@ -0,0 +1,289 @@ +# Identity + +You are an AI assistant that helps users improve their Texera workflows. Your task is to analyze workflow prompts and generate structured suggestions that can enhance or correct the workflow execution. + +# Instructions + +* The LLM will receive the prompt as a **single JSON object string** with the following top-level keys: + * `intention`: a string that states what the user wants to accomplish. If the user did not provide one this will be the default string "Recommend improvements and fixes of current workflows". + * `focusingOperators`: an array of operator information that the user explicitly wants you to pay extra attention to. This list may be empty. + * `workflowInterpretation`: the workflow interpretation itself, which can be either + * single workflow object containing links and operators with input schema and error(if any) + * single workflow object containing links operators with input schema and error(if any), PLUS list of paths (each path is a list of operator ID) + + In other words, the old RAW / BY_PATH objects are now nested inside `workflowInterpretation`, together with the additional context keys shown above. + +* Your goal is to generate a list of actionable suggestions. Each suggestion must: + - Be expressed clearly in natural language. + - Include a structured JSON object describing the required changes. + +* Your suggestion should align with user's intention. Either: + - Help users **fix** potential issues in their workflow (e.g., broken links, misconfigured operators, incorrect data flow) + - **Improve** their workflow by adding useful steps (e.g., for data cleaning, exploratory data analysis, data visualization, AI/ML model training or inference). + +* Each suggestion must include: + - A `suggestion` string that explains the proposed improvement. Should be high level + - A `suggestionType` field with one of two values: `"fix"` or `"improve"`. + - A `changes` object containing: + * `operatorsToAdd`: array of new or updated operators with ID, type, and properties. + * When you want to update an operator, please put it in this array, making sure the operatorID is the same with the original operatorID + * For available operator types and their format, you should do a search in the operator_format.json file in the knowledge base. + * `linksToAdd`: array of new links with operator ID and port info. + * You must make sure the operatorID in the each link exists either in given workflow json, or in the new operator list + * `operatorsToDelete`: list of operator IDs to remove. + * `linksToDelete`: list of link IDs to remove. + +* Do not include extra explanation or commentary. Your response must be a valid JSON objects. It will be parsed automatically. + +* Here are available operator types and their descriptions +- IntervalJoin: Join two inputs with left table join key in the range of [right table join key, right table join key + constant value] +- DotPlot: Visualize data using a dot plot +- CartesianProduct: Append fields together to get the cartesian product of two inputs +- HuggingFaceSentimentAnalysis: Analyzing Sentiments with a Twitter-Based Model from Hugging Face +- PythonUDFSourceV2: User-defined function operator in Python script +- TwitterFullArchiveSearch: Retrieve data from Twitter Full Archive Search API +- SklearnLogisticRegressionCV: Sklearn Logistic Regression Cross Validation Operator +- JSONLFileScan: Scan data from a JSONL file +- CandlestickChart: Visualize data in a Candlestick Chart +- ReservoirSampling: Reservoir Sampling with k items being kept randomly +- ScatterMatrixChart: Visualize datasets in a Scatter Matrix +- SklearnKNN: Sklearn K-nearest Neighbors Operator +- SklearnProbabilityCalibration: Sklearn Probability Calibration Operator +- SortPartitions: Sort Partitions +- DumbbellPlot: Visualize data in a Dumbbell Plots. A dumbbell plots (also known as a lollipop chart) is typically used to compare two distinct values or time points for the same entity. +- If: If +- SklearnSDG: Sklearn Stochastic Gradient Descent Operator +- URLVisualizer: Render the content of URL +- Dummy: A dummy operator used as a placeholder. +- HuggingFaceTextSummarization: Summarize the given text content with a mini2bert pre-trained model from Hugging Face +- Union: Unions the output rows from multiple input operators +- SklearnGradientBoosting: Sklearn Gradient Boosting Operator +- KNNRegressorTrainer: Sklearn KNN Regressor Operator +- RUDFSource: User-defined function operator in R script +- HuggingFaceIrisLogisticRegression: Predict whether an iris is an Iris-setosa using a pre-trained logistic regression model +- ContinuousErrorBands: Visualize error or uncertainty along a continuous line +- TwitterSearch: Retrieve data from Twitter Search API +- SklearnPassiveAggressive: Sklearn Passive Aggressive Operator +- HTMLVisualizer: Render the result of HTML content +- SklearnComplementNaiveBayes: Sklearn Complement Naive Bayes Operator +- URLFetcher: Fetch the content of a single url +- JavaUDF: User-defined function operator in Java script +- PieChart: Visualize data in a Pie Chart +- DictionaryMatcher: Matches tuples if they appear in a given dictionary +- UnnestString: Unnest the string values in the column separated by a delimiter to multiple values +- BubbleChart: a 3D Scatter Plot; Bubbles are graphed using x and y labels, and their sizes determined by a z-value. +- RedditSearch: Search for recent posts with python-wrapped Reddit API, PRAW +- SVRTrainer: Sklearn SVM Regressor Operator +- RUDF: User-defined function operator in R script +- BoxViolinPlot: Visualize data using either a Box Plot or a Violin Plot. Box plots are drawn as a box with a vertical line down the middle which is mean value, and has horizontal lines attached to each side (known as "whiskers"). Violin plots provide more detail by showing a smoothed density curve on each side, and also include a box plot inside for comparison. +- SklearnAdaptiveBoosting: Sklearn Adaptive Boosting Operator +- Scatterplot: View the result in a scatterplot +- SklearnPerceptron: Sklearn Linear Perceptron Operator +- KeywordSearch: Search for keyword(s) in a string column +- PythonUDFV2: User-defined function operator in Python script +- SklearnLogisticRegression: Sklearn Logistic Regression Operator +- SklearnRandomForest: Sklearn Random Forest Operator +- TypeCasting: Cast between types +- SklearnGaussianNaiveBayes: Sklearn Gaussian Naive Bayes Operator +- AsterixDBSource: Read data from a AsterixDB instance +- DualInputPortsPythonUDFV2: User-defined function operator in Python script +- Histogram: Visualize data in a Histogram Chart +- SklearnDummyClassifier: Sklearn Dummy Classifier Operator +- Distinct: Remove duplicate tuples +- NetworkGraph: Visualize data in a network graph +- WaterfallChart: Visualize data as a waterfall chart +- Limit: Limit the number of output rows +- Scorer: Scorer for machine learning models +- SklearnExtraTrees: Sklearn Extra Trees Operator +- FileScan: Scan data from a file +- GanttChart: A Gantt chart is a type of bar chart that illustrates a project schedule. The chart lists the tasks to be performed on the vertical axis, and time intervals on the horizontal axis. The width of the horizontal bars in the graph shows the duration of each activity. +- TernaryPlot: Points are graphed on a Ternary Plot using 3 specified data fields +- SVCTrainer: Sklearn SVM Classifier Operator +- SklearnLinearRegression: Sklearn Linear Regression Operator +- MySQLSource: Read data from a MySQL instance +- CSVOldFileScan: Scan data from a CSVOld file +- CSVFileScan: Scan data from a CSV file +- FunnelPlot: Visualize data in a Funnel Plot +- Projection: Keeps or drops the column +- Filter: Performs a filter operation +- SklearnRidge: Sklearn Ridge Regression Operator +- Intersect: Take the intersect of two inputs +- SklearnPrediction: Skleanr Prediction Operator +- SymmetricDifference: find the symmetric difference (the set of elements which are in either of the sets, but not in their intersection) of two inputs +- FigureFactoryTable: Visualize data in a figure factory table +- FilledAreaPlot: Visualize data in filled area plot +- SklearnRidgeCV: Sklearn Ridge Regression Cross Validation Operator +- IcicleChart: Visualize hierarchical data from root to leaves +- Regex: Search a regular expression in a string column +- HeatMap: Visualize data in a HeatMap Chart +- TablesPlot: Visualize data in a table chart. +- HierarchyChart: Visualize data in hierarchy +- SklearnExtraTree: Sklearn Extra Tree Operator +- Sort: Sort based on the columns and sorting methods +- Scatter3DChart: Visualize data in a Scatter3D Plot +- SklearnBagging: Sklearn Bagging Operator +- Difference: find the set difference of two inputs +- ContourPlot: Displays terrain or gradient variations in a Contour Plot +- PythonLambdaFunction: Modify or add a new column with more ease +- WordCloud: Generate word cloud for texts +- LineChart: View the result in line chart +- RandomKSampling: random sampling with given percentage +- Split: Split data to two different ports +- SklearnMultiLayerPerceptron: Sklearn Multi-layer Perceptron Operator +- BarChart: Visualize data in a Bar Chart +- HashJoin: join two inputs +- PythonTableReducer: Reduce Table to Tuple +- Dendrogram: Visualize data in a Dendrogram +- KNNClassifierTrainer: Sklearn KNN Classifier Operator +- SklearnMultinomialNaiveBayes: Sklearn Multinomial Naive Bayes Operator +- SklearnDecisionTree: Sklearn Decision Tree Operator +- PostgreSQLSource: Read data from a PostgreSQL instance +- ArrowSource: Scan data from a Arrow file +- SankeyDiagram: Visualize data using a Sankey diagram +- SklearnSVM: Sklearn Support Vector Machine Operator +- SklearnBernoulliNaiveBayes: Sklearn Bernoulli Naive Bayes Operator +- ImageVisualizer: visualize image content +- SklearnLinearSVM: Sklearn Linear Support Vector Machine Operator +- TextInput: Source data from manually inputted text +- HuggingFaceSpamSMSDetection: Spam Detection by SMS Spam Detection Model from Hugging Face +- QuiverPlot: Visualize vector data in a Quiver Plot +- SklearnNearestCentroid: Sklearn Nearest Centroid Operator +- Aggregate: Calculate different types of aggregation values + +# Requirement +* For available operator types, you MUST use the given operator types based on their description + * A function call tool of getting the json schemas is given, you MUST use it to get the json schema of the operators you want to add/modify +* When you want to update the existing operators, you MUST put it in the `changes.operatorsToAdd` array, making sure the operatorID is the same with the original operatorID +* When deleting the operators, you MUST make sure the operatorIDs exist in the given workflow json. +* When adding the links, you MUST make sure the operatorID in each link exists either in given workflow json, or in the new operator list +* When deleting the links, you MUST make sure the linkIDs exist in the given workflow json. +* suggestion field in each suggestion should be high level. You do NOT need to explain the detail like add `X` after `Y`. +* When the intention contains the specific operations on which operator to use, you MUST follow the intention to check that operator's json schema and generate that operator with link to the existing operator. + +# Guideline of using PythonUDFV2 operator +PythonUDFV2: performs the customized data cleaning logic. There are 2 APIs to process the data in different units. +1. Tuple API. + +```python +from pytexera import * + +class ProcessTupleOperator(UDFOperatorV2): + + def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]: + yield tuple_ + +``` +* Tuple API takes one input tuple from a port at a time. It returns an iterator of optional `TupleLike` instances. A `TupleLike` is any data structure that supports key-value pairs, such as `pytexera.Tuple`, `dict`, `defaultdict`, `NamedTuple`, etc. +* Tuple API is useful for implementing functional operations which are applied to tuples one by one, such as map, reduce, and filter. + +2. Table API. + +```python +from pytexera import * + +class ProcessTableOperator(UDFTableOperator): + + def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]: + yield table +``` +* Table API consumes a `Table` at a time, which consists of all the whole table from a port. It returns an iterator of optional `TableLike` instances. A `TableLike ` is a collection of `TupleLike`, and currently, we support `pytexera.Table` and `pandas.DataFrame` as a `TableLike` instance. +* Table API is useful for implementing blocking operations that will consume the whole column to do operations. + +* Here are some examples of using two APIs: + * Example 1: use Tuple API to normalize the `state` column to standard uppercase US state code: +```python +from pytexera import * +class ProcessTupleOperator(UDFOperatorV2): + """ + Standardise free-form state names/abbreviations to two-letter codes. + Unknown values are upper-cased unchanged. + """ + def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]: + _STATE_MAP = { + "california": "CA", "ca": "CA", + "new york": "NY", "ny": "NY", + "texas": "TX", "tx": "TX", + } + if tuple_["BILLINGCOMPANYCODE"] is not None: + raw = str(tuple_["BILLINGCOMPANYCODE"]).strip().lower() + tuple_["BILLINGCOMPANYCODE"] = _STATE_MAP.get(raw, raw.upper()) + yield tuple_ +``` + * Example 2: use Table API to convert the `CREATIONTIME` column into a uniform format +```python +from pytexera import * +import pandas as pd +import re + +class ProcessTableOperator(UDFTableOperator): + """ + Clean the string‐typed `CREATIONTIME` column **without** converting the column + to real datetimes: + + 1. Normalise every entry to the canonical string format YYYY-MM-DD. + 2. Drop rows that cannot be interpreted as a valid calendar date + (remain as `None` after normalisation). + """ + + _YMD = re.compile(r"^\s*(\d{4})[/-](\d{1,2})[/-](\d{1,2}).*$") # 2024-5-7 or 2024/05/07 + _MDY = re.compile(r"^\s*(\d{1,2})[/-](\d{1,2})[/-](\d{4}).*$") # 5/7/2024 or 05-07-2024 + _DMY = re.compile(r"^\s*(\d{1,2})[/-](\d{1,2})[/-](\d{4}).*$") # 07-05-2024 (day first) + + @staticmethod + def _pad(n: str) -> str: + """Left-pad month/day to two digits.""" + return n.zfill(2) + + @classmethod + def _normalise(cls, s: str): + """ + Convert a date string to YYYY-MM-DD. Returns None if no pattern matches + or values are out of range (very light validation). + """ + if not isinstance(s, str): + return None + + s = s.strip() + m = cls._YMD.match(s) + if m: + y, mm, dd = m.groups() + else: + m = cls._MDY.match(s) + if m: + mm, dd, y = m.groups() + else: + m = cls._DMY.match(s) + if m: + dd, mm, y = m.groups() + else: + return None # unrecognised format + + # Basic sanity check + try: + year = int(y) + month = int(mm) + day = int(dd) + if not (1 <= month <= 12 and 1 <= day <= 31): + return None + except ValueError: + return None + + return f"{year:04d}-{cls._pad(str(month))}-{cls._pad(str(day))}" + + def process_table(self, table: Table, port: int): + df: pd.DataFrame = table + + # Apply pure-string normalisation + df["CREATIONTIME"] = df["CREATIONTIME"].apply(self._normalise) + + yield df +``` + +* When writing the udf code, you MUST NOT change the class name +* You should import pandas, numpy, sklearn and other common python packages when you want too use them +* You don't need to import typing for the type annotations. +* Tuple you can think it as the 'Dict' type. You should ONLY use `[]` to do tuple's field's read & write. DO NOT use methods like `tuple_.get()` or `tuple_.set()` or `tuple_.values()` +* Table you can think it as the `pandas.Dataframe`. +* Try to use `yield` to return the result tuple or result df; DO NOT have the code that will yield tuple /table multiple times +* When doing tuple / table APIs, be aware of the None value for the given key, e.g. `tuple_['key']` could potentially be None value. +* DO NOT cast types for any values in the tuple or table \ No newline at end of file diff --git a/core/suggestion-service/files/instruction_for_sanitizor.md b/core/suggestion-service/files/instruction_for_sanitizor.md new file mode 100644 index 00000000000..92d5e5d654b --- /dev/null +++ b/core/suggestion-service/files/instruction_for_sanitizor.md @@ -0,0 +1,95 @@ +# Identity + +You are an AI assistant that sanitizes workflow suggestions for Texera. +Your task is to ensure the operators proposed in the suggestions have correct and complete operatorProperties that match their expected JSON schema. + +# Instructions + +* You will receive a **SuggestionSanitization** object containing: + - A **suggestions** field, which is a list of suggestions to sanitize. + - A **schemas** field, which is a list of operator schemas describing the correct jsonSchema for each operatorType. + +* Your goal is to sanitize the suggestion list to ensure: + - For each operator to add (in `operatorsToAdd` field): + - Its `operatorProperties` must match the expected fields and types based on its `operatorType`'s `jsonSchema`. + - If any required property is missing, you must fill it with its default value from the `jsonSchema`. + - If no default is provided, fill missing fields with a reasonable empty value (e.g., `""` for string, `0` for number, `[]` for array, `{}` for object, `false` for boolean). + - Remove any unknown properties that are not defined in the schema. + - **Do not** modify `operatorID`, `operatorType`, `customDisplayName`, `linksToAdd`, or `operatorsToDelete` unless necessary. + - Preserve the original high-level `suggestion` string and `suggestionType`. + +* Your final output must be: + - A valid **SuggestionList** object. + - Strictly follow the JSON schema provided (no missing fields, no extra fields). + +* If no change is necessary for a suggestion, output it as is. +* Do not explain, comment, or summarize your actions. Only return the corrected JSON output. + +# Tips + +* Always validate operatorProperties based on the correct `operatorType` and its provided `jsonSchema`. +* Fill or correct fields **deeply** if they are nested (e.g., nested objects, arrays). +* Assume that if a field in the `jsonSchema` has `"nullable": true`, it is allowed to be `null` if no value is provided. +* Definitions referenced with `$ref` must be properly resolved. + +# Examples + + +{ + "suggestions": { + "suggestions": [ + { + "suggestion": "Improve data visualization by adding a line chart", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "LineChart", + "operatorID": "linechart-123", + "operatorProperties": { + "xLabel": "X Axis", + "lines": [] + } + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + } + ] + }, + "schemas": [ + { + "operatorType": "LineChart", + "jsonSchema": { ... } + } + ] +} + + + +{ + "suggestions": [ + { + "suggestion": "Improve data visualization by adding a line chart", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "LineChart", + "operatorID": "linechart-123", + "operatorProperties": { + "dummyPropertyList": [], + "yLabel": "Y Axis", + "xLabel": "X Axis", + "lines": [] + } + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + } + ] +} + \ No newline at end of file diff --git a/core/suggestion-service/files/instruction_for_suggestion.md b/core/suggestion-service/files/instruction_for_suggestion.md new file mode 100644 index 00000000000..fd346fac122 --- /dev/null +++ b/core/suggestion-service/files/instruction_for_suggestion.md @@ -0,0 +1,158 @@ +# Identity + +You are an AI assistant that helps users improve their Texera workflows. Your task is to analyze workflow prompts and generate structured suggestions that can enhance or correct the workflow execution. + +# Instructions + +* The LLM will receive the prompt as a **single JSON object string** with the following top-level keys: + * `intention`: a string that states what the user wants to accomplish. If the user did not provide one this will be the default string "Recommend improvements and fixes of current workflows". + * `focusingOperators`: an array of operator information that the user explicitly wants you to pay extra attention to. This list may be empty. + * `workflowInterpretation`: the workflow interpretation itself, which can be either + * single workflow object containing links and operators with input schema and error(if any) + * single workflow object containing links operators with input schema and error(if any), PLUS list of paths (each path is a list of operator ID) + +* Your goal is to generate a list of actionable suggestions. Each suggestion must: + - Be expressed clearly in natural language. + - Include a structured JSON object describing the required changes. + +* Your suggestion should align with user's intention. Either: + - Help users **fix** potential issues in their workflow (e.g., broken links, misconfigured operators, incorrect data flow) + - **Improve** their workflow by adding useful steps (e.g., for data cleaning, exploratory data analysis, data visualization, AI/ML model training or inference). + +* Each suggestion must include: + - A `suggestion` string that explains the proposed improvement. Should be high level + - A `suggestionType` field with one of two values: `"fix"` or `"improve"`. + - A `changes` object containing: + * `operatorsToAdd`: array of new or updated operators with ID, type, and properties. + * When you want to update an operator, please put it in this array, making sure the operatorID is the same with the original operatorID + * For available operator types and their format, you should do a search in the operator_format.json file in the knowledge base. + * `linksToAdd`: array of new links with operator ID and port info. + * You must make sure the operatorID in the each link exists either in given workflow json, or in the new operator list + * `operatorsToDelete`: list of operator IDs to remove. + +* Do not include extra explanation or commentary. Your response must be a valid JSON objects. It will be parsed automatically. + +# Available Operator Types and Descriptions +- IntervalJoin: Join two inputs with left table join key in the range of [right table join key, right table join key + constant value] +- DotPlot: Visualize data using a dot plot +- CartesianProduct: Append fields together to get the cartesian product of two inputs +- HuggingFaceSentimentAnalysis: Analyzing Sentiments with a Twitter-Based Model from Hugging Face +- PythonUDFSourceV2: User-defined function operator in Python script +- TwitterFullArchiveSearch: Retrieve data from Twitter Full Archive Search API +- SklearnLogisticRegressionCV: Sklearn Logistic Regression Cross Validation Operator +- JSONLFileScan: Scan data from a JSONL file +- CandlestickChart: Visualize data in a Candlestick Chart +- ReservoirSampling: Reservoir Sampling with k items being kept randomly +- ScatterMatrixChart: Visualize datasets in a Scatter Matrix +- SklearnKNN: Sklearn K-nearest Neighbors Operator +- SklearnProbabilityCalibration: Sklearn Probability Calibration Operator +- SortPartitions: Sort Partitions +- DumbbellPlot: Visualize data in a Dumbbell Plots. A dumbbell plots (also known as a lollipop chart) is typically used to compare two distinct values or time points for the same entity. +- If: If +- SklearnSDG: Sklearn Stochastic Gradient Descent Operator +- URLVisualizer: Render the content of URL +- Dummy: A dummy operator used as a placeholder. +- HuggingFaceTextSummarization: Summarize the given text content with a mini2bert pre-trained model from Hugging Face +- Union: Unions the output rows from multiple input operators +- SklearnGradientBoosting: Sklearn Gradient Boosting Operator +- KNNRegressorTrainer: Sklearn KNN Regressor Operator +- RUDFSource: User-defined function operator in R script +- HuggingFaceIrisLogisticRegression: Predict whether an iris is an Iris-setosa using a pre-trained logistic regression model +- ContinuousErrorBands: Visualize error or uncertainty along a continuous line +- TwitterSearch: Retrieve data from Twitter Search API +- SklearnPassiveAggressive: Sklearn Passive Aggressive Operator +- HTMLVisualizer: Render the result of HTML content +- SklearnComplementNaiveBayes: Sklearn Complement Naive Bayes Operator +- URLFetcher: Fetch the content of a single url +- JavaUDF: User-defined function operator in Java script +- PieChart: Visualize data in a Pie Chart +- DictionaryMatcher: Matches tuples if they appear in a given dictionary +- UnnestString: Unnest the string values in the column separated by a delimiter to multiple values +- BubbleChart: a 3D Scatter Plot; Bubbles are graphed using x and y labels, and their sizes determined by a z-value. +- RedditSearch: Search for recent posts with python-wrapped Reddit API, PRAW +- SVRTrainer: Sklearn SVM Regressor Operator +- RUDF: User-defined function operator in R script +- BoxViolinPlot: Visualize data using either a Box Plot or a Violin Plot. Box plots are drawn as a box with a vertical line down the middle which is mean value, and has horizontal lines attached to each side (known as "whiskers"). Violin plots provide more detail by showing a smoothed density curve on each side, and also include a box plot inside for comparison. +- SklearnAdaptiveBoosting: Sklearn Adaptive Boosting Operator +- Scatterplot: View the result in a scatterplot +- SklearnPerceptron: Sklearn Linear Perceptron Operator +- KeywordSearch: Search for keyword(s) in a string column +- PythonUDFV2: User-defined function operator in Python script +- SklearnLogisticRegression: Sklearn Logistic Regression Operator +- SklearnRandomForest: Sklearn Random Forest Operator +- TypeCasting: Cast between types +- SklearnGaussianNaiveBayes: Sklearn Gaussian Naive Bayes Operator +- AsterixDBSource: Read data from a AsterixDB instance +- DualInputPortsPythonUDFV2: User-defined function operator in Python script +- Histogram: Visualize data in a Histogram Chart +- SklearnDummyClassifier: Sklearn Dummy Classifier Operator +- Distinct: Remove duplicate tuples +- NetworkGraph: Visualize data in a network graph +- WaterfallChart: Visualize data as a waterfall chart +- Limit: Limit the number of output rows +- Scorer: Scorer for machine learning models +- SklearnExtraTrees: Sklearn Extra Trees Operator +- FileScan: Scan data from a file +- GanttChart: A Gantt chart is a type of bar chart that illustrates a project schedule. The chart lists the tasks to be performed on the vertical axis, and time intervals on the horizontal axis. The width of the horizontal bars in the graph shows the duration of each activity. +- TernaryPlot: Points are graphed on a Ternary Plot using 3 specified data fields +- SVCTrainer: Sklearn SVM Classifier Operator +- SklearnLinearRegression: Sklearn Linear Regression Operator +- MySQLSource: Read data from a MySQL instance +- CSVOldFileScan: Scan data from a CSVOld file +- CSVFileScan: Scan data from a CSV file +- FunnelPlot: Visualize data in a Funnel Plot +- Projection: Keeps or drops the column +- Filter: Performs a filter operation +- SklearnRidge: Sklearn Ridge Regression Operator +- Intersect: Take the intersect of two inputs +- SklearnPrediction: Skleanr Prediction Operator +- SymmetricDifference: find the symmetric difference (the set of elements which are in either of the sets, but not in their intersection) of two inputs +- FigureFactoryTable: Visualize data in a figure factory table +- FilledAreaPlot: Visualize data in filled area plot +- SklearnRidgeCV: Sklearn Ridge Regression Cross Validation Operator +- IcicleChart: Visualize hierarchical data from root to leaves +- Regex: Search a regular expression in a string column +- HeatMap: Visualize data in a HeatMap Chart +- TablesPlot: Visualize data in a table chart. +- HierarchyChart: Visualize data in hierarchy +- SklearnExtraTree: Sklearn Extra Tree Operator +- Sort: Sort based on the columns and sorting methods +- Scatter3DChart: Visualize data in a Scatter3D Plot +- SklearnBagging: Sklearn Bagging Operator +- Difference: find the set difference of two inputs +- ContourPlot: Displays terrain or gradient variations in a Contour Plot +- PythonLambdaFunction: Modify or add a new column with more ease +- WordCloud: Generate word cloud for texts +- LineChart: View the result in line chart +- RandomKSampling: random sampling with given percentage +- Split: Split data to two different ports +- SklearnMultiLayerPerceptron: Sklearn Multi-layer Perceptron Operator +- BarChart: Visualize data in a Bar Chart +- HashJoin: join two inputs +- PythonTableReducer: Reduce Table to Tuple +- Dendrogram: Visualize data in a Dendrogram +- KNNClassifierTrainer: Sklearn KNN Classifier Operator +- SklearnMultinomialNaiveBayes: Sklearn Multinomial Naive Bayes Operator +- SklearnDecisionTree: Sklearn Decision Tree Operator +- PostgreSQLSource: Read data from a PostgreSQL instance +- ArrowSource: Scan data from a Arrow file +- SankeyDiagram: Visualize data using a Sankey diagram +- SklearnSVM: Sklearn Support Vector Machine Operator +- SklearnBernoulliNaiveBayes: Sklearn Bernoulli Naive Bayes Operator +- ImageVisualizer: visualize image content +- SklearnLinearSVM: Sklearn Linear Support Vector Machine Operator +- TextInput: Source data from manually inputted text +- HuggingFaceSpamSMSDetection: Spam Detection by SMS Spam Detection Model from Hugging Face +- QuiverPlot: Visualize vector data in a Quiver Plot +- SklearnNearestCentroid: Sklearn Nearest Centroid Operator +- Aggregate: Calculate different types of aggregation values + +# Tips +* For available operator types and their format, you MUST do a search in the operator_json_schema.json file in the knowledge base to know the json format of the operator you want to recommend +* When you want to update the existing operators, you MUST put it in the `changes.operatorsToAdd` array, making sure the operatorID is the same with the original operatorID +* You can ONLY use operator types are in the following set: +``` +['WordCloud', 'SymmetricDifference', 'SklearnSVM', 'RUDFSource', 'TablesPlot', 'RedditSearch', 'HTMLVisualizer', 'SklearnPerceptron', 'WaterfallChart', 'KNNRegressorTrainer', 'RUDF', 'SklearnRidgeCV', 'MySQLSource', 'DictionaryMatcher', 'If', 'SklearnPassiveAggressive', 'Difference', 'KNNClassifierTrainer', 'DumbbellPlot', 'URLFetcher', 'HuggingFaceSpamSMSDetection', 'SklearnNearestCentroid', 'Limit', 'FilledAreaPlot', 'Scatterplot', 'SklearnProbabilityCalibration', 'BarChart', 'SVCTrainer', 'SklearnPrediction', 'ReservoirSampling', 'SklearnRandomForest', 'SklearnMultiLayerPerceptron', 'Dendrogram', 'ArrowSource', 'SklearnGradientBoosting', 'FileScan', 'SklearnDummyClassifier', 'UnnestString', 'Split', 'ContinuousErrorBands', 'CSVOldFileScan', 'SortPartitions', 'HeatMap', 'TwitterSearch', 'QuiverPlot', 'SklearnDecisionTree', 'URLVisualizer', 'Scatter3DChart', 'HuggingFaceIrisLogisticRegression', 'SklearnAdaptiveBoosting', 'DualInputPortsPythonUDFV2', 'TernaryPlot', 'Filter', 'IntervalJoin', 'DotPlot', 'SklearnKNN', 'TextInput', 'SklearnLinearRegression', 'JavaUDF', 'SklearnLinearSVM', 'ContourPlot', 'Sort', 'SklearnGaussianNaiveBayes', 'HuggingFaceTextSummarization', 'HierarchyChart', 'RandomKSampling', 'GanttChart', 'CSVFileScan', 'SklearnComplementNaiveBayes', 'FigureFactoryTable', 'BoxViolinPlot', 'ScatterMatrixChart', 'SklearnExtraTree', 'PythonLambdaFunction', 'SklearnExtraTrees', 'Union', 'TwitterFullArchiveSearch', 'JSONLFileScan', 'SklearnSDG', 'NetworkGraph', 'FunnelPlot', 'SklearnLogisticRegressionCV', 'SVRTrainer', 'IcicleChart', 'LineChart', 'SklearnLogisticRegression', 'PieChart', 'PythonUDFV2', 'ImageVisualizer', 'Regex', 'PythonTableReducer', 'HuggingFaceSentimentAnalysis', 'CandlestickChart', 'KeywordSearch', 'SankeyDiagram', 'Intersect', 'PostgreSQLSource', 'Distinct', 'CartesianProduct', 'HashJoin', 'SklearnBagging', 'BubbleChart', 'AsterixDBSource', 'SklearnMultinomialNaiveBayes', 'Projection', 'SklearnRidge', 'SklearnBernoulliNaiveBayes', 'PythonUDFSourceV2', 'Aggregate', 'TypeCasting', 'Histogram', 'Scorer', 'Dummy'] +``` +* When adding the links, you MUST make sure the operatorID in the each link exists either in given workflow json, or in the new operator list +* suggestion field in each suggestion should be high level. You do NOT need to explain the detail like add `X` after `Y`. diff --git a/core/suggestion-service/files/operator_format.json b/core/suggestion-service/files/operator_format.json new file mode 100644 index 00000000000..ec6c4f6208b --- /dev/null +++ b/core/suggestion-service/files/operator_format.json @@ -0,0 +1,4153 @@ +[ + { + "operatorID": "IntervalJoin-operator-94a405e8-f1d0-44c7-a024-dc6c355ef389", + "operatorType": "IntervalJoin", + "operatorVersion": "N/A", + "operatorProperties": { + "constant": 10, + "includeLeftBound": true, + "includeRightBound": true, + "timeIntervalType": "day", + "dummyPropertyList": [], + "leftAttributeName": null, + "rightAttributeName": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "left table", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "right table", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Interval Join" + }, + { + "operatorID": "DotPlot-operator-073e4322-fd56-4f39-a7cc-fd9e1bf01dd6", + "operatorType": "DotPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Count Attribute": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dot Plot" + }, + { + "operatorID": "CartesianProduct-operator-8c355f9a-625e-41ef-b9a4-faa58a49fc42", + "operatorType": "CartesianProduct", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "left", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "right", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Cartesian Product" + }, + { + "operatorID": "HuggingFaceSentimentAnalysis-operator-057df3f1-bf65-4569-a529-67022a473cc8", + "operatorType": "HuggingFaceSentimentAnalysis", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "attribute": null, + "Positive result attribute": "huggingface_sentiment_positive", + "Neutral result attribute": "huggingface_sentiment_neutral", + "Negative result attribute": "huggingface_sentiment_negative" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hugging Face Sentiment Analysis" + }, + { + "operatorID": "PythonUDFSourceV2-operator-5b79ff6d-5f36-434d-ad9c-1d65c2dd0638", + "operatorType": "PythonUDFSourceV2", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "# from pytexera import *\n# class GenerateOperator(UDFSourceOperator):\n# \n# @overrides\n# \n# def produce(self) -> Iterator[Union[TupleLike, TableLike, None]]:\n# yield\n", + "workers": 1, + "columns": [] + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "1-out Python UDF" + }, + { + "operatorID": "TwitterFullArchiveSearch-operator-abf6914f-f6f1-409d-b2f1-7910270bddab", + "operatorType": "TwitterFullArchiveSearch", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "apiKey": null, + "apiSecretKey": null, + "stopWhenRateLimited": false, + "searchQuery": null, + "fromDateTime": "2021-04-01T00:00:00Z", + "toDateTime": "2021-05-01T00:00:00Z", + "limit": 100 + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Twitter Full Archive Search API" + }, + { + "operatorID": "SklearnLogisticRegressionCV-operator-a7c38481-03c4-41f4-b496-623bd48d8817", + "operatorType": "SklearnLogisticRegressionCV", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Logistic Regression Cross Validation" + }, + { + "operatorID": "JSONLFileScan-operator-4da5d4a2-a94c-439b-9ab2-9c8c69ec10aa", + "operatorType": "JSONLFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fileName": null, + "fileEncoding": "UTF_8", + "limit": null, + "offset": null, + "flatten": null + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "JSONL File Scan" + }, + { + "operatorID": "CandlestickChart-operator-61822092-5166-489a-beff-79ee44ff07f6", + "operatorType": "CandlestickChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "date": null, + "open": null, + "high": null, + "low": null, + "close": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Candlestick Chart" + }, + { + "operatorID": "ReservoirSampling-operator-be6bb046-5c14-452d-8284-f399c4368175", + "operatorType": "ReservoirSampling", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "number of item sampled in reservoir sampling": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Reservoir Sampling" + }, + { + "operatorID": "ScatterMatrixChart-operator-6bb2ebfa-cb71-4927-b97f-5d5e57c831c6", + "operatorType": "ScatterMatrixChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Selected Attributes": [], + "Color": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Scatter Matrix Chart" + }, + { + "operatorID": "SklearnKNN-operator-ae1b0bf4-cee7-43b2-999f-8c6ade032e00", + "operatorType": "SklearnKNN", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "K-nearest Neighbors" + }, + { + "operatorID": "SklearnProbabilityCalibration-operator-1d1d950a-7d55-4261-b476-deb5725e5d39", + "operatorType": "SklearnProbabilityCalibration", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Probability Calibration" + }, + { + "operatorID": "SortPartitions-operator-570f7cdd-c55f-4051-a1d0-5411892c27ed", + "operatorType": "SortPartitions", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "sortAttributeName": null, + "domainMin": null, + "domainMax": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sort Partitions" + }, + { + "operatorID": "DumbbellPlot-operator-40a538d5-bd1d-4ba5-a731-372a749617df", + "operatorType": "DumbbellPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "categoryColumnName": null, + "dumbbellStartValue": null, + "dumbbellEndValue": null, + "measurementColumnName": null, + "comparedColumnName": null, + "dots": [], + "showLegends": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dumbbell Plot" + }, + { + "operatorID": "If-operator-bff0e021-2948-4503-a98f-6ae4ab6334b5", + "operatorType": "If", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "conditionName": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "Condition", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "False", + "allowMultiInputs": false, + "isDynamicPort": false + }, + { + "portID": "output-1", + "displayName": "True", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "If" + }, + { + "operatorID": "SklearnSDG-operator-279f8e8e-1a02-44e5-a2ec-3e0e928428fb", + "operatorType": "SklearnSDG", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Stochastic Gradient Descent" + }, + { + "operatorID": "URLVisualizer-operator-a3280fb2-19fe-404a-891f-bdc1dec5603e", + "operatorType": "URLVisualizer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "urlContentAttrName": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "URL Visualizer" + }, + { + "operatorID": "Dummy-operator-101e4a24-e75c-4820-84dc-838042549310", + "operatorType": "Dummy", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "dummyOperator": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dummy" + }, + { + "operatorID": "HuggingFaceTextSummarization-operator-fb0fcd94-0a72-4fb5-af74-52189fab70b2", + "operatorType": "HuggingFaceTextSummarization", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "attribute": null, + "Result attribute name": "summary" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hugging Face Text Summarization" + }, + { + "operatorID": "Union-operator-3c22307d-c6f2-4e84-929e-efccac4531af", + "operatorType": "Union", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Union" + }, + { + "operatorID": "SklearnGradientBoosting-operator-148ef602-52b9-4b37-bf1e-beab517c4781", + "operatorType": "SklearnGradientBoosting", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Gradient Boosting" + }, + { + "operatorID": "KNNRegressorTrainer-operator-f5419456-e879-4510-8b6c-a4c3ed322f0e", + "operatorType": "KNNRegressorTrainer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "paraList": [], + "groundTruthAttribute": null, + "Selected Features": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "parameter", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "KNN Regressor" + }, + { + "operatorID": "RUDFSource-operator-30130996-ae93-4e58-9df3-3f1fbc92012d", + "operatorType": "RUDFSource", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "# If using Table API:\n# function() { \n# return (data.frame(Column_Here = \"Value_Here\")) \n# }\n\n# If using Tuple API:\n# library(coro)\n# coro::generator(function() {\n# yield (list(text= \"hello world!\"))\n# })", + "workers": 1, + "useTupleAPI": false, + "columns": [] + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "1-out R UDF" + }, + { + "operatorID": "HuggingFaceIrisLogisticRegression-operator-dbdeaa54-b343-4941-b83e-1df81a971300", + "operatorType": "HuggingFaceIrisLogisticRegression", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "petalLengthCmAttribute": null, + "petalWidthCmAttribute": null, + "prediction class name": "Species_prediction", + "prediction probability name": "Species_probability" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hugging Face Iris Logistic Regression" + }, + { + "operatorID": "ContinuousErrorBands-operator-dc2ff6db-6181-42b9-b071-300e10a76c9c", + "operatorType": "ContinuousErrorBands", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "xLabel": "X Axis", + "yLabel": "Y Axis", + "bands": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Continuous Error Bands" + }, + { + "operatorID": "TwitterSearch-operator-4eba20a9-718c-445b-b573-6a6a8523e139", + "operatorType": "TwitterSearch", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "apiKey": null, + "apiSecretKey": null, + "stopWhenRateLimited": false, + "searchQuery": null, + "limit": 100 + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Twitter Search API" + }, + { + "operatorID": "SklearnPassiveAggressive-operator-a296a825-31d2-419c-ab82-d5fcaefca355", + "operatorType": "SklearnPassiveAggressive", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Passive Aggressive" + }, + { + "operatorID": "HTMLVisualizer-operator-0e1e767c-b5f2-49d1-a518-9cf44eb3d8c8", + "operatorType": "HTMLVisualizer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "htmlContentAttrName": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "HTML Visualizer" + }, + { + "operatorID": "SklearnComplementNaiveBayes-operator-03bf8714-500d-4564-9420-5b2f197c93cc", + "operatorType": "SklearnComplementNaiveBayes", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Complement Naive Bayes" + }, + { + "operatorID": "URLFetcher-operator-bd715acc-9ecc-41e3-88f2-4c5921afe6fc", + "operatorType": "URLFetcher", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "url": null, + "decodingMethod": null + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "URL fetcher" + }, + { + "operatorID": "JavaUDF-operator-53281828-0900-4954-a2e4-4e6a13baba8f", + "operatorType": "JavaUDF", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "import edu.uci.ics.texera.workflow.common.operators.map.MapOpExec;\nimport edu.uci.ics.amber.engine.common.model.tuple.Tuple;\nimport edu.uci.ics.amber.engine.common.model.tuple.TupleLike;\nimport scala.Function1;\nimport java.io.Serializable;\n\npublic class JavaUDFOpExec extends MapOpExec {\n public JavaUDFOpExec () {\n this.setMapFunc((Function1 & Serializable) this::processTuple);\n }\n \n public TupleLike processTuple(Tuple tuple) {\n return tuple;\n }\n}", + "workers": 1, + "retainInputColumns": true, + "outputColumns": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Java UDF" + }, + { + "operatorID": "PieChart-operator-5202bee8-24bb-429b-ad3d-9cf2e43ba3f0", + "operatorType": "PieChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "value": null, + "name": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Pie Chart" + }, + { + "operatorID": "DictionaryMatcher-operator-d822ff6f-edc4-4f71-b94c-b3e4582ae55f", + "operatorType": "DictionaryMatcher", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Dictionary": null, + "Attribute": null, + "result attribute": "matched", + "Matching type": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dictionary matcher" + }, + { + "operatorID": "UnnestString-operator-e5941278-400e-440c-86d5-8bdd04a6df21", + "operatorType": "UnnestString", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Delimiter": ",", + "Attribute": null, + "Result attribute": "unnestResult" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Unnest String" + }, + { + "operatorID": "BubbleChart-operator-cc325164-2231-4c1e-81d3-f357685263cd", + "operatorType": "BubbleChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "xValue": null, + "yValue": null, + "zValue": null, + "enableColor": false, + "colorCategory": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Bubble Chart" + }, + { + "operatorID": "RedditSearch-operator-0b36dafa-e6c6-497f-aec8-83fc034f09f9", + "operatorType": "RedditSearch", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "clientId": null, + "clientSecret": null, + "query": null, + "limit": 100, + "sorting": "none" + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Reddit Search" + }, + { + "operatorID": "SVRTrainer-operator-2311a515-dcd9-4ec1-aecc-0654df6f2124", + "operatorType": "SVRTrainer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "paraList": [], + "groundTruthAttribute": null, + "Selected Features": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "parameter", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "SVM Regressor" + }, + { + "operatorID": "RUDF-operator-7aad59ec-5ee3-4242-b3cd-b43c8a1b2f54", + "operatorType": "RUDF", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "# If using Table API:\n# function(table, port) { \n# return (table) \n# }\n\n# If using Tuple API:\n# library(coro)\n# coro::generator(function(tuple, port) {\n# yield (tuple)\n# })", + "workers": 1, + "useTupleAPI": false, + "retainInputColumns": true, + "outputColumns": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "R UDF" + }, + { + "operatorID": "BoxViolinPlot-operator-afb8bcda-2f85-4bee-97b8-354958272047", + "operatorType": "BoxViolinPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "value": null, + "Quartile Method": "linear", + "horizontalOrientation": false, + "violinPlot": false, + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Box/Violin Plot" + }, + { + "operatorID": "SklearnAdaptiveBoosting-operator-6732949a-95a4-42d2-8493-41799da0166c", + "operatorType": "SklearnAdaptiveBoosting", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Adaptive Boosting" + }, + { + "operatorID": "Scatterplot-operator-ac841e0c-7016-4934-bfc6-d566373fc6cf", + "operatorType": "Scatterplot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "xColumn": null, + "yColumn": null, + "colorColumn": null, + "xLogScale": false, + "yLogScale": false, + "hoverName": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Scatter Plot" + }, + { + "operatorID": "SklearnPerceptron-operator-a4a1b83f-072b-4723-ab60-527a1f3a0083", + "operatorType": "SklearnPerceptron", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Linear Perceptron" + }, + { + "operatorID": "KeywordSearch-operator-c7f3b41f-e12d-4025-b0b1-0ef75aac817e", + "operatorType": "KeywordSearch", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "attribute": null, + "keyword": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Keyword Search" + }, + { + "operatorID": "PythonUDFV2-operator-f051c7e2-c487-47b7-a151-7efe9ca78874", + "operatorType": "PythonUDFV2", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "# Choose from the following templates:\n# \n# from pytexera import *\n# \n# class ProcessTupleOperator(UDFOperatorV2):\n# \n# @overrides\n# def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n# yield tuple_\n# \n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "workers": 1, + "retainInputColumns": true, + "outputColumns": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Python UDF" + }, + { + "operatorID": "SklearnLogisticRegression-operator-e7b0a305-9954-4a79-9008-3b2a1edb34e6", + "operatorType": "SklearnLogisticRegression", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Logistic Regression" + }, + { + "operatorID": "SklearnRandomForest-operator-bc418f9c-1798-4d74-903d-fbe5c7af2a6d", + "operatorType": "SklearnRandomForest", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Random Forest" + }, + { + "operatorID": "TypeCasting-operator-c3b17263-69b0-448f-a23f-21bba9301bd7", + "operatorType": "TypeCasting", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "typeCastingUnits": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Type Casting" + }, + { + "operatorID": "SklearnGaussianNaiveBayes-operator-546fc1da-1fff-44a4-b2cb-49dd02485fd8", + "operatorType": "SklearnGaussianNaiveBayes", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Gaussian Naive Bayes" + }, + { + "operatorID": "AsterixDBSource-operator-99ff1b64-4443-43b2-9919-d7bb0c86b978", + "operatorType": "AsterixDBSource", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "host": null, + "port": "default", + "database": null, + "table": null, + "limit": null, + "offset": null, + "keywordSearch": false, + "keywordSearchByColumn": null, + "keywords": null, + "progressive": false, + "batchByColumn": null, + "min": "auto", + "max": "auto", + "interval": 1000000000, + "geoSearch": false, + "geoSearchByColumns": [], + "geoSearchBoundingBox": [], + "regexSearch": false, + "regexSearchByColumn": null, + "regex": null, + "filterCondition": false, + "predicates": [] + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "AsterixDB Source" + }, + { + "operatorID": "DualInputPortsPythonUDFV2-operator-08949588-ed84-48e4-b1d5-427d155c2bf7", + "operatorType": "DualInputPortsPythonUDFV2", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "code": "# Choose from the following templates:\n# \n# from pytexera import *\n# \n# class ProcessTupleOperator(UDFOperatorV2):\n# \n# @overrides\n# def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n# yield tuple_\n# \n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "workers": 1, + "retainInputColumns": true, + "outputColumns": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "model", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "tuples", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "2-in Python UDF" + }, + { + "operatorID": "Histogram-operator-09a3d8a7-96f4-4e27-9d17-b851e4e9b9c2", + "operatorType": "Histogram", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "color": null, + "separateBy": null, + "marginal": null, + "pattern": null, + "value": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Histogram" + }, + { + "operatorID": "SklearnDummyClassifier-operator-c09cb6cf-edf2-49eb-a3f6-269f11baeab7", + "operatorType": "SklearnDummyClassifier", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dummy Classifier" + }, + { + "operatorID": "Distinct-operator-a50aeabd-61ca-419b-bbf1-476dac6c5f10", + "operatorType": "Distinct", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Distinct" + }, + { + "operatorID": "NetworkGraph-operator-55d6f6e5-443b-4a32-b61d-84ac9ffe0c17", + "operatorType": "NetworkGraph", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "source": null, + "destination": null, + "title": "Network Graph" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Network Graph" + }, + { + "operatorID": "WaterfallChart-operator-fde6eb29-7620-4364-b52e-c483bbaf94eb", + "operatorType": "WaterfallChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "xColumn": null, + "yColumn": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Waterfall Chart" + }, + { + "operatorID": "Limit-operator-51368085-289f-4c81-9710-7f50c773986c", + "operatorType": "Limit", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "limit": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Limit" + }, + { + "operatorID": "Scorer-operator-7c7678b4-6808-4b88-b668-827a4b239953", + "operatorType": "Scorer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "isRegression": false, + "actualValueColumn": null, + "predictValueColumn": null, + "classificationFlag": [], + "regressionFlag": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Machine Learning Scorer" + }, + { + "operatorID": "SklearnExtraTrees-operator-51253af5-d50e-4f7a-9b88-6a982a122aeb", + "operatorType": "SklearnExtraTrees", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Extra Trees" + }, + { + "operatorID": "FileScan-operator-9c176295-3e96-4b0d-96ae-4414af9c4156", + "operatorType": "FileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fileName": null, + "encoding": "UTF_8", + "extract": false, + "outputFileName": false, + "attributeType": "string", + "attributeName": "line", + "fileScanLimit": null, + "fileScanOffset": null + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": " File Scan" + }, + { + "operatorID": "GanttChart-operator-17fd05af-1037-4104-82f7-031c450edf79", + "operatorType": "GanttChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "pattern": null, + "start": null, + "finish": null, + "task": null, + "color": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Gantt Chart" + }, + { + "operatorID": "TernaryPlot-operator-204e4579-92f5-4560-a19c-31fbcbaf8be7", + "operatorType": "TernaryPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "firstVariable": null, + "secondVariable": null, + "thirdVariable": null, + "colorEnabled": false, + "colorDataField": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Ternary Plot" + }, + { + "operatorID": "SVCTrainer-operator-8368d270-ce0f-499c-bd00-9685ae9e6eed", + "operatorType": "SVCTrainer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "paraList": [], + "groundTruthAttribute": null, + "Selected Features": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "parameter", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "SVM Classifier" + }, + { + "operatorID": "SklearnLinearRegression-operator-4a2f1610-bae4-46b7-9b7e-477a5b076cfe", + "operatorType": "SklearnLinearRegression", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "degree": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Linear Regression" + }, + { + "operatorID": "MySQLSource-operator-8967246d-8668-4746-8fcd-d4623d72514d", + "operatorType": "MySQLSource", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "host": null, + "port": "default", + "database": null, + "table": null, + "username": null, + "password": null, + "limit": null, + "offset": null, + "keywordSearch": false, + "keywordSearchByColumn": null, + "keywords": null, + "progressive": false, + "batchByColumn": null, + "min": "auto", + "max": "auto", + "interval": 1000000000 + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "MySQL Source" + }, + { + "operatorID": "CSVOldFileScan-operator-bbdf2a26-1246-48ab-93a1-5c52ac9e3d84", + "operatorType": "CSVOldFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fileName": null, + "fileEncoding": "UTF_8", + "limit": null, + "offset": null, + "customDelimiter": ",", + "hasHeader": true + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "CSVOld File Scan" + }, + { + "operatorID": "CSVFileScan-operator-5bd48e8e-8eea-4750-b23a-870c8687e374", + "operatorType": "CSVFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fileName": null, + "fileEncoding": "UTF_8", + "limit": null, + "offset": null, + "customDelimiter": ",", + "hasHeader": true + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "CSV File Scan" + }, + { + "operatorID": "FunnelPlot-operator-f16667f4-84a3-4aca-b977-2358362379bc", + "operatorType": "FunnelPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "x": null, + "y": null, + "color": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Funnel Plot" + }, + { + "operatorID": "Projection-operator-c7071d17-f212-4f15-b25c-4e61b235c56a", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "isDrop": false, + "attributes": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection" + }, + { + "operatorID": "Filter-operator-66cf7145-0659-487b-9c81-8e4dffb2f44a", + "operatorType": "Filter", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "predicates": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Filter" + }, + { + "operatorID": "SklearnRidge-operator-5792371d-fe13-4862-9eb5-570f04de6dbc", + "operatorType": "SklearnRidge", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Ridge Regression" + }, + { + "operatorID": "Intersect-operator-c1a5cf03-631a-4300-b096-cad673322c95", + "operatorType": "Intersect", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Intersect" + }, + { + "operatorID": "SklearnPrediction-operator-55d13d3a-da51-4d9b-a3eb-4ac6973551e4", + "operatorType": "SklearnPrediction", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Model Attribute": "model", + "Output Attribute Name": "prediction", + "Ground Truth Attribute Name to Ignore": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "model", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sklearn Prediction" + }, + { + "operatorID": "SymmetricDifference-operator-32ce4cc8-9da8-4e9e-9065-65df7b12c410", + "operatorType": "SymmetricDifference", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "SymmetricDifference" + }, + { + "operatorID": "FigureFactoryTable-operator-bdb0630f-8515-4867-9a50-69592b225e92", + "operatorType": "FigureFactoryTable", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fontSize": null, + "fontColor": null, + "rowHeight": null, + "add attribute": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Figure Factory Table" + }, + { + "operatorID": "FilledAreaPlot-operator-8a20c87e-8a37-4086-9cfb-338aa143f4cd", + "operatorType": "FilledAreaPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "x": null, + "y": null, + "lineGroup": null, + "color": null, + "facetColumn": null, + "pattern": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Filled Area Plot" + }, + { + "operatorID": "SklearnRidgeCV-operator-29d4bbec-1ef5-45b9-a2db-acd5695db037", + "operatorType": "SklearnRidgeCV", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Ridge Regression Cross Validation" + }, + { + "operatorID": "IcicleChart-operator-4c8cfb86-c602-4665-95dc-d419d0fa7d37", + "operatorType": "IcicleChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "hierarchy": [], + "value": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Icicle Chart" + }, + { + "operatorID": "Regex-operator-e908abb1-7c91-41ac-97dd-13f23e9b9455", + "operatorType": "Regex", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "caseInsensitive": false, + "attribute": null, + "regex": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Regular Expression" + }, + { + "operatorID": "HeatMap-operator-2dd1c4f2-f275-4c7e-a175-0048a4a114e6", + "operatorType": "HeatMap", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "x": null, + "y": null, + "Values": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Heatmap" + }, + { + "operatorID": "TablesPlot-operator-aaa3c914-5727-4c8a-8d6a-d8942b732234", + "operatorType": "TablesPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "add attribute": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Tables Plot" + }, + { + "operatorID": "HierarchyChart-operator-b8980f15-0f01-481c-af20-a3cf506ef058", + "operatorType": "HierarchyChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "hierarchyChartType": null, + "hierarchy": [], + "value": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hierarchy Chart" + }, + { + "operatorID": "SklearnExtraTree-operator-f707f26b-d288-4072-8c7b-7eae4c2cf51a", + "operatorType": "SklearnExtraTree", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Extra Tree" + }, + { + "operatorID": "Sort-operator-78396fbf-85d4-40ff-b0f0-7ed3328b78cf", + "operatorType": "Sort", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "attributes": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sort" + }, + { + "operatorID": "Scatter3DChart-operator-34866a13-48cd-49d1-ab77-252e0a7963e2", + "operatorType": "Scatter3DChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "x": null, + "y": null, + "z": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Scatter3D Chart" + }, + { + "operatorID": "SklearnBagging-operator-de64f368-2abc-4871-89e8-b07c04978885", + "operatorType": "SklearnBagging", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Bagging" + }, + { + "operatorID": "Difference-operator-18e513a2-b20f-46ea-8459-fb5b79a8ae09", + "operatorType": "Difference", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "left", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "right", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Difference" + }, + { + "operatorID": "ContourPlot-operator-fbc683a2-b8fb-4e9a-ae60-ff4040477c30", + "operatorType": "ContourPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "gridSize": "10", + "connectGaps": true, + "x": null, + "y": null, + "z": null, + "Coloring Method": "heatmap" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Contour Plot" + }, + { + "operatorID": "PythonLambdaFunction-operator-e96035f2-8bef-41e2-8b3d-665d381513e1", + "operatorType": "PythonLambdaFunction", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "lambdaAttributeUnits": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Python Lambda Function" + }, + { + "operatorID": "WordCloud-operator-1f6f0630-a12f-4dde-8f0d-1da1099793a6", + "operatorType": "WordCloud", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "textColumn": null, + "topN": 100 + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Word Cloud" + }, + { + "operatorID": "LineChart-operator-0e6f35eb-b550-41be-955c-44412c8c57ff", + "operatorType": "LineChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "yLabel": "Y Axis", + "xLabel": "X Axis", + "lines": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Line Chart" + }, + { + "operatorID": "RandomKSampling-operator-b144d179-9567-447b-9c87-dbe737b369f0", + "operatorType": "RandomKSampling", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "random k sample percentage": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Random K Sampling" + }, + { + "operatorID": "Split-operator-640cebe2-ad96-4302-9c38-ddd8632b1f61", + "operatorType": "Split", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "k": 80, + "random": true, + "seed": 1 + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + }, + { + "portID": "output-1", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Split" + }, + { + "operatorID": "SklearnMultiLayerPerceptron-operator-a4c22712-a10a-413c-a0a8-34fcb8316905", + "operatorType": "SklearnMultiLayerPerceptron", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Multi-layer Perceptron" + }, + { + "operatorID": "BarChart-operator-4a666711-5089-468a-aca3-5fa3c954c6ae", + "operatorType": "BarChart", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fields": null, + "categoryColumn": "No Selection", + "horizontalOrientation": false, + "pattern": null, + "value": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Bar Chart" + }, + { + "operatorID": "HashJoin-operator-bfc31771-261a-4c38-8df8-ec8525f5c0f7", + "operatorType": "HashJoin", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "buildAttributeName": null, + "probeAttributeName": null, + "joinType": "inner" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "left", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "right", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hash Join" + }, + { + "operatorID": "PythonTableReducer-operator-d60f85bf-7357-41f4-a843-587c0eaff39e", + "operatorType": "PythonTableReducer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "lambdaAttributeUnits": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Python Table Reducer" + }, + { + "operatorID": "Dendrogram-operator-682e5623-7f7a-4d19-908d-76807af1bd40", + "operatorType": "Dendrogram", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "threshold": null, + "xVal": null, + "yVal": null, + "Labels": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Dendrogram" + }, + { + "operatorID": "KNNClassifierTrainer-operator-4b23afa3-25c8-4b0f-b1b9-88a20f59d005", + "operatorType": "KNNClassifierTrainer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "paraList": [], + "groundTruthAttribute": null, + "Selected Features": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "parameter", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "KNN Classifier" + }, + { + "operatorID": "SklearnMultinomialNaiveBayes-operator-175a52d9-5ac1-4e7c-90a1-ab830965c4cb", + "operatorType": "SklearnMultinomialNaiveBayes", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Multinomial Naive Bayes" + }, + { + "operatorID": "SklearnDecisionTree-operator-d209040c-fb3a-4831-8a6f-bc632fb16bb2", + "operatorType": "SklearnDecisionTree", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Decision Tree" + }, + { + "operatorID": "PostgreSQLSource-operator-acc0043b-0948-473a-963f-e3c51f250d77", + "operatorType": "PostgreSQLSource", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "host": null, + "port": "default", + "database": null, + "table": null, + "username": null, + "password": null, + "limit": null, + "offset": null, + "keywordSearch": false, + "keywordSearchByColumn": null, + "keywords": null, + "progressive": false, + "batchByColumn": null, + "min": "auto", + "max": "auto", + "interval": 1000000000 + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "PostgreSQL Source" + }, + { + "operatorID": "ArrowSource-operator-2b69b043-3b33-4064-91df-5e11c42e01b5", + "operatorType": "ArrowSource", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "fileName": null, + "limit": null, + "offset": null + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Arrow File Scan" + }, + { + "operatorID": "SankeyDiagram-operator-045b0d37-a678-405d-b7dd-09a6866afbbf", + "operatorType": "SankeyDiagram", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "Source Attribute": null, + "Target Attribute": null, + "Value Attribute": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sankey Diagram" + }, + { + "operatorID": "SklearnSVM-operator-05ad5a59-59a1-4e4e-bc25-eb698ec02b72", + "operatorType": "SklearnSVM", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Support Vector Machine" + }, + { + "operatorID": "SklearnBernoulliNaiveBayes-operator-2f502786-1c9b-4062-8d45-c2bb07e253c5", + "operatorType": "SklearnBernoulliNaiveBayes", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Bernoulli Naive Bayes" + }, + { + "operatorID": "ImageVisualizer-operator-4f3ab694-0c74-4ae0-a79b-df5b2568f4cd", + "operatorType": "ImageVisualizer", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "binaryContent": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Image Visualizer" + }, + { + "operatorID": "SklearnLinearSVM-operator-74638340-5cbf-4b0e-862f-ba2adb37d744", + "operatorType": "SklearnLinearSVM", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Linear Support Vector Machine" + }, + { + "operatorID": "TextInput-operator-94c6b3cd-009b-4373-adbd-3b460c0b3825", + "operatorType": "TextInput", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "textInput": null, + "attributeType": "string", + "attributeName": "line", + "fileScanLimit": null, + "fileScanOffset": null + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Text Input" + }, + { + "operatorID": "HuggingFaceSpamSMSDetection-operator-74a33cc6-3d8e-4c2a-8e98-bbb64b0ee907", + "operatorType": "HuggingFaceSpamSMSDetection", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "attribute": null, + "Spam result attribute": "is_spam", + "Score result attribute": "score" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Hugging Face Spam Detection" + }, + { + "operatorID": "QuiverPlot-operator-9556e857-9496-4428-9c4b-4c863c45a704", + "operatorType": "QuiverPlot", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "x": null, + "y": null, + "u": null, + "v": null + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Quiver Plot" + }, + { + "operatorID": "SklearnNearestCentroid-operator-a243c2f9-645f-459f-8ea0-16ae505a527e", + "operatorType": "SklearnNearestCentroid", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "target": null, + "countVectorizer": false, + "text": null, + "tfidfTransformer": false + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "training", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + }, + { + "portID": "input-1", + "displayName": "testing", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Nearest Centroid" + }, + { + "operatorID": "Aggregate-operator-f6eb5dc3-f10a-4b98-ac81-4a2f63cb9b39", + "operatorType": "Aggregate", + "operatorVersion": "N/A", + "operatorProperties": { + "dummyPropertyList": [], + "aggregations": [], + "groupByKeys": [] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Aggregate" + } +] \ No newline at end of file diff --git a/core/suggestion-service/files/operator_json_schema.json b/core/suggestion-service/files/operator_json_schema.json new file mode 100644 index 00000000000..cbb9333d898 --- /dev/null +++ b/core/suggestion-service/files/operator_json_schema.json @@ -0,0 +1,41270 @@ +{ + "operators": [ + { + "operatorType": "IntervalJoin", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "leftAttributeName": { + "enum": [ + "integer", + "long", + "double", + "timestamp" + ] + }, + "rightAttributeName": { + "const": { + "$data": "leftAttributeName" + } + } + }, + "properties": { + "constant": { + "propertyOrder": 1, + "type": "integer", + "default": 10, + "description": "left attri in (right, right + constant)", + "title": "Interval Constant" + }, + "includeLeftBound": { + "propertyOrder": 2, + "type": "boolean", + "default": true, + "description": "Include condition left attri = right attri", + "title": "Include Left Bound" + }, + "includeRightBound": { + "propertyOrder": 3, + "type": "boolean", + "default": true, + "description": "Include condition left attri = right attri", + "title": "Include Right Bound" + }, + "timeIntervalType": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "default": "day", + "enum": [ + "year", + "month", + "day", + "hour", + "minute", + "second" + ], + "description": "Year, Month, Day, Hour, Minute or Second", + "title": "Time interval type" + }, + "dummyPropertyList": { + "propertyOrder": 5, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "leftAttributeName": { + "propertyOrder": 8, + "type": "string", + "description": "Choose one attribute in the left table", + "title": "Left Input attr", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "rightAttributeName": { + "propertyOrder": 9, + "type": "string", + "description": "Choose one attribute in the right table", + "title": "Right Input attr", + "autofill": "attributeName", + "autofillAttributeOnPort": 1 + } + }, + "required": [ + "constant", + "includeLeftBound", + "includeRightBound", + "leftAttributeName", + "rightAttributeName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "IntervalJoin" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Interval Join", + "operatorDescription": "Join two inputs with left table join key in the range of [right table join key, right table join key + constant value]", + "operatorGroupName": "Join", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "left table", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "right table", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "DotPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Count Attribute": { + "propertyOrder": 6, + "type": "string", + "description": "the attribute for the counting of the dot plot", + "title": "Count Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "Count Attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "DotPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dot Plot", + "operatorDescription": "Visualize data using a dot plot", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "CartesianProduct", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "CartesianProduct" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Cartesian Product", + "operatorDescription": "Append fields together to get the cartesian product of two inputs", + "operatorGroupName": "Join", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "left", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "right", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HuggingFaceSentimentAnalysis", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "attribute": { + "propertyOrder": 6, + "type": "string", + "description": "column to perform sentiment analysis on", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Positive result attribute": { + "propertyOrder": 7, + "type": "string", + "default": "huggingface_sentiment_positive", + "description": "column name of the sentiment analysis result (positive)", + "title": "Positive result attribute" + }, + "Neutral result attribute": { + "propertyOrder": 8, + "type": "string", + "default": "huggingface_sentiment_neutral", + "description": "column name of the sentiment analysis result (neutral)", + "title": "Neutral result attribute" + }, + "Negative result attribute": { + "propertyOrder": 9, + "type": "string", + "default": "huggingface_sentiment_negative", + "description": "column name of the sentiment analysis result (negative)", + "title": "Negative result attribute" + } + }, + "required": [ + "attribute", + "Positive result attribute", + "Neutral result attribute", + "Negative result attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HuggingFaceSentimentAnalysis" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hugging Face Sentiment Analysis", + "operatorDescription": "Analyzing Sentiments with a Twitter-Based Model from Hugging Face", + "operatorGroupName": "Hugging Face", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PythonUDFSourceV2", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "# from pytexera import *\n# class GenerateOperator(UDFSourceOperator):\n# \n# @overrides\n# \n# def produce(self) -> Iterator[Union[TupleLike, TableLike, None]]:\n# yield\n", + "description": "Input your code here", + "title": "Python script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to launch", + "title": "Worker count" + }, + "columns": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "The columns of the source", + "title": "Columns" + } + }, + "required": [ + "code", + "workers" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PythonUDFSourceV2" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "1-out Python UDF", + "operatorDescription": "User-defined function operator in Python script", + "operatorGroupName": "Python", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TwitterFullArchiveSearch", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "apiKey": { + "propertyOrder": 4, + "type": "string", + "title": "API Key" + }, + "apiSecretKey": { + "propertyOrder": 5, + "type": "string", + "title": "API Secret Key" + }, + "stopWhenRateLimited": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "description": "Stop when hitting rate limit?", + "title": "Stop Upon Rate Limit" + }, + "searchQuery": { + "propertyOrder": 7, + "type": "string", + "description": "Up to 1024 characters (Limited By Twitter)", + "title": "Search Query", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "fromDateTime": { + "propertyOrder": 8, + "type": "string", + "default": "2021-04-01T00:00:00Z", + "description": "ISO 8601 format", + "title": "From Datetime" + }, + "toDateTime": { + "propertyOrder": 9, + "type": "string", + "default": "2021-05-01T00:00:00Z", + "description": "ISO 8601 format", + "title": "To Datetime" + }, + "limit": { + "propertyOrder": 10, + "type": "integer", + "default": 100, + "description": "Maximum number of tweets to retrieve", + "title": "Limit" + } + }, + "required": [ + "apiKey", + "apiSecretKey", + "stopWhenRateLimited", + "searchQuery", + "fromDateTime", + "toDateTime", + "limit" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TwitterFullArchiveSearch" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Twitter Full Archive Search API", + "operatorDescription": "Retrieve data from Twitter Full Archive Search API", + "operatorGroupName": "External API", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnLogisticRegressionCV", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnLogisticRegressionCV" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Logistic Regression Cross Validation", + "operatorDescription": "Sklearn Logistic Regression Cross Validation Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "JSONLFileScan", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fileName": { + "propertyOrder": 4, + "type": "string", + "title": "File" + }, + "fileEncoding": { + "propertyOrder": 5, + "type": "string", + "default": "UTF_8", + "enum": [ + "UTF_8", + "UTF_16", + "US_ASCII" + ], + "description": "decoding charset to use on input", + "title": "File Encoding" + }, + "limit": { + "propertyOrder": 6, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 7, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "flatten": { + "propertyOrder": 8, + "type": "boolean", + "description": "flatten nested objects and arrays", + "title": "Flatten" + } + }, + "required": [ + "fileName", + "fileEncoding", + "flatten" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "JSONLFileScan" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "JSONL File Scan", + "operatorDescription": "Scan data from a JSONL file", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "CandlestickChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "date": { + "propertyOrder": 6, + "type": "string", + "description": "the date of the candlestick", + "title": "Date Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "open": { + "propertyOrder": 7, + "type": "string", + "description": "the opening price of the candlestick", + "title": "Opening Price Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "high": { + "propertyOrder": 8, + "type": "string", + "description": "the highest price of the candlestick", + "title": "Highest Price Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "low": { + "propertyOrder": 9, + "type": "string", + "description": "the lowest price of the candlestick", + "title": "Lowest Price Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "close": { + "propertyOrder": 10, + "type": "string", + "description": "the closing price of the candlestick", + "title": "Closing Price Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "date", + "open", + "high", + "low", + "close" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "CandlestickChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Candlestick Chart", + "operatorDescription": "Visualize data in a Candlestick Chart", + "operatorGroupName": "Financial", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ReservoirSampling", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "number of item sampled in reservoir sampling": { + "propertyOrder": 6, + "type": "integer", + "description": "reservoir sampling with k items being kept randomly", + "title": "Number of item sampled in reservoir sampling" + } + }, + "required": [ + "number of item sampled in reservoir sampling" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ReservoirSampling" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Reservoir Sampling", + "operatorDescription": "Reservoir Sampling with k items being kept randomly", + "operatorGroupName": "Utilities", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ScatterMatrixChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Selected Attributes": { + "propertyOrder": 6, + "type": "array", + "items": { + "type": "string" + }, + "description": "The axes of each scatter plot in the matrix.", + "title": "Selected Attributes", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + }, + "Color": { + "propertyOrder": 7, + "type": "string", + "description": "Column to color points", + "title": "Color Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "Selected Attributes", + "Color" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ScatterMatrixChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Scatter Matrix Chart", + "operatorDescription": "Visualize datasets in a Scatter Matrix", + "operatorGroupName": "Statistical", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnKNN", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnKNN" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "K-nearest Neighbors", + "operatorDescription": "Sklearn K-nearest Neighbors Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnProbabilityCalibration", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnProbabilityCalibration" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Probability Calibration", + "operatorDescription": "Sklearn Probability Calibration Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SortPartitions", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "sortAttributeName": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "sortAttributeName": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute to sort (must be numerical).", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "domainMin": { + "propertyOrder": 5, + "type": "integer", + "description": "Minimum value of the domain of the attribute.", + "title": "Attribute Domain Min" + }, + "domainMax": { + "propertyOrder": 6, + "type": "integer", + "description": "Maximum value of the domain of the attribute.", + "title": "Attribute Domain Max" + } + }, + "required": [ + "sortAttributeName", + "domainMin", + "domainMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SortPartitions" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Sort Partitions", + "operatorDescription": "Sort Partitions", + "operatorGroupName": "Sort", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "DumbbellPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "measurementColumnName": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "categoryColumnName": { + "propertyOrder": 6, + "type": "string", + "description": "the name of the category column", + "title": "Category Column Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "dumbbellStartValue": { + "propertyOrder": 7, + "type": "string", + "description": "the start point value of each dumbbell", + "title": "Dumbbell Start Value" + }, + "dumbbellEndValue": { + "propertyOrder": 8, + "type": "string", + "description": "the end value of each dumbbell", + "title": "Dumbbell End Value" + }, + "measurementColumnName": { + "propertyOrder": 9, + "type": "string", + "description": "the name of the measurement column", + "title": "Measurement Column Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "comparedColumnName": { + "propertyOrder": 10, + "type": "string", + "description": "the column name that is being compared", + "title": "Compared Column Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "dots": { + "propertyOrder": 11, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DumbbellDotConfig" + }, + "title": "Dots" + }, + "showLegends": { + "propertyOrder": 12, + "type": "boolean", + "description": "whether show legends in the graph", + "title": "Show Legends?" + } + }, + "required": [ + "categoryColumnName", + "dumbbellStartValue", + "dumbbellEndValue", + "measurementColumnName", + "comparedColumnName", + "showLegends" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "DumbbellPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "DumbbellDotConfig": { + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "dot": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dot": { + "propertyOrder": 1, + "type": "string", + "description": "value for dot axis", + "title": "Dot Column Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "dot" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dumbbell Plot", + "operatorDescription": "Visualize data in a Dumbbell Plots. A dumbbell plots (also known as a lollipop chart) is typically used to compare two distinct values or time points for the same entity.", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "If", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "conditionName": { + "propertyOrder": 4, + "type": "string", + "description": "name of the state variable to evaluate", + "title": "Condition State" + } + }, + "required": [ + "conditionName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "If" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "If", + "operatorDescription": "If", + "operatorGroupName": "Control Block", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "Condition", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "False", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "True", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnSDG", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnSDG" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Stochastic Gradient Descent", + "operatorDescription": "Sklearn Stochastic Gradient Descent Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "URLVisualizer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "urlContentAttrName": { + "enum": [ + "string" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "urlContentAttrName": { + "propertyOrder": 4, + "type": "string", + "title": "URL content", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "urlContentAttrName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "URLVisualizer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "URL Visualizer", + "operatorDescription": "Render the content of URL", + "operatorGroupName": "Media", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Dummy", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "dummyOperator": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "description": "The description of this dummy operator", + "title": "Description" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Dummy" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dummy", + "operatorDescription": "A dummy operator used as a placeholder.", + "operatorGroupName": "Utilities", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": true, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "supportReconfiguration": true, + "allowPortCustomization": true + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HuggingFaceTextSummarization", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "attribute": { + "propertyOrder": 6, + "type": "string", + "description": "attribute to perform text summarization on", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Result attribute name": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "default": "summary", + "description": "attribute name of the text summary result", + "title": "Result attribute name" + } + }, + "required": [ + "attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HuggingFaceTextSummarization" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hugging Face Text Summarization", + "operatorDescription": "Summarize the given text content with a mini2bert pre-trained model from Hugging Face", + "operatorGroupName": "Hugging Face", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Union", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Union" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Union", + "operatorDescription": "Unions the output rows from multiple input operators", + "operatorGroupName": "Set", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": true, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnGradientBoosting", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnGradientBoosting" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Gradient Boosting", + "operatorDescription": "Sklearn Gradient Boosting Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "KNNRegressorTrainer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "paraList": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/HyperParameters(SklearnAdvancedKNNParameters)" + }, + "title": "Parameter Setting" + }, + "groundTruthAttribute": { + "propertyOrder": 5, + "type": "string", + "description": "Ground truth attribute column", + "title": "Ground Truth Attribute Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Selected Features": { + "propertyOrder": 8, + "type": "array", + "items": { + "type": "string" + }, + "description": "Features used to train the model", + "title": "Selected Features", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "paraList", + "groundTruthAttribute", + "Selected Features" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "KNNRegressorTrainer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HyperParameters(SklearnAdvancedKNNParameters)": { + "type": "object", + "additionalProperties": false, + "properties": { + "parameter": { + "propertyOrder": 1, + "type": "string", + "enum": [ + "n_neighbors", + "p", + "weights", + "algorithm", + "leaf_size", + "metric", + "metric_params" + ], + "description": "Choose the name of the parameter", + "title": "Parameter" + }, + "parametersSource": { + "propertyOrder": 2, + "type": "boolean", + "default": false, + "description": "Parameter from workflow", + "title": "Workflow" + }, + "attribute": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 1 + }, + "value": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "true", + "hideOnNull": true + } + }, + "required": [ + "parameter", + "parametersSource" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "KNN Regressor", + "operatorDescription": "Sklearn KNN Regressor Operator", + "operatorGroupName": "Advanced Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "parameter", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "RUDFSource", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "# If using Table API:\n# function() { \n# return (data.frame(Column_Here = \"Value_Here\")) \n# }\n\n# If using Tuple API:\n# library(coro)\n# coro::generator(function() {\n# yield (list(text= \"hello world!\"))\n# })", + "description": "Input your code here", + "title": "R Source UDF Script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to launch", + "title": "Worker count" + }, + "useTupleAPI": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "description": "Check this box to use Tuple API, leave unchecked to use Table API", + "title": "Use Tuple API?" + }, + "columns": { + "propertyOrder": 7, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "The columns of the source", + "title": "Columns" + } + }, + "required": [ + "code", + "workers", + "useTupleAPI" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "RUDFSource" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "1-out R UDF", + "operatorDescription": "User-defined function operator in R script", + "operatorGroupName": "R", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HuggingFaceIrisLogisticRegression", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "petalLengthCmAttribute": { + "propertyOrder": 6, + "type": "string", + "description": "attribute in your dataset corresponding to PetalLengthCm", + "title": "Petal Length Cm Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "petalWidthCmAttribute": { + "propertyOrder": 7, + "type": "string", + "description": "attribute in your dataset corresponding to PetalWidthCm", + "title": "Petal Width Cm Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "prediction class name": { + "propertyOrder": 8, + "type": "string", + "default": "Species_prediction", + "description": "output attribute name for the predicted class of species", + "title": "Prediction class name" + }, + "prediction probability name": { + "propertyOrder": 9, + "type": "string", + "default": "Species_probability", + "description": "output attribute name for the prediction's probability of being a Iris-setosa", + "title": "Prediction probability name" + } + }, + "required": [ + "petalLengthCmAttribute", + "petalWidthCmAttribute", + "prediction class name", + "prediction probability name" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HuggingFaceIrisLogisticRegression" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hugging Face Iris Logistic Regression", + "operatorDescription": "Predict whether an iris is an Iris-setosa using a pre-trained logistic regression model", + "operatorGroupName": "Hugging Face", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ContinuousErrorBands", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "xLabel": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "default": "X Axis", + "description": "Label used for x axis", + "title": "X Label" + }, + "yLabel": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "default": "Y Axis", + "description": "Label used for y axis", + "title": "Y Label" + }, + "bands": { + "propertyOrder": 8, + "type": "array", + "items": { + "$ref": "#/definitions/BandConfig" + }, + "title": "Bands" + } + }, + "required": [ + "bands" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ContinuousErrorBands" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "BandConfig": { + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "yValue": { + "enum": [ + "integer", + "long", + "double" + ] + }, + "xValue": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "yUpper": { + "propertyOrder": 1, + "type": "string", + "description": "Represents upper bound error of y-values", + "title": "Y-Axis Upper Bound", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "yLower": { + "propertyOrder": 2, + "type": "string", + "description": "Represents lower bound error of y-values", + "title": "Y-Axis Lower Bound", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "fillColor": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "description": "must be a valid CSS color or hex color string", + "title": "Fill Color" + }, + "y": { + "propertyOrder": 4, + "type": "string", + "description": "value for y axis", + "title": "Y Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "x": { + "propertyOrder": 5, + "type": "string", + "description": "value for x axis", + "title": "X Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "mode": { + "propertyOrder": 6, + "type": "string", + "default": "line with dots", + "enum": [ + "line", + "dots", + "line with dots" + ], + "title": "Line Mode" + }, + "name": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "title": "Line Name" + }, + "color": { + "propertyOrder": 8, + "nullable": true, + "type": "string", + "description": "must be a valid CSS color or hex color string", + "title": "Line Color" + } + }, + "required": [ + "yUpper", + "yLower", + "y", + "x", + "mode" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Continuous Error Bands", + "operatorDescription": "Visualize error or uncertainty along a continuous line", + "operatorGroupName": "Statistical", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TwitterSearch", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "apiKey": { + "propertyOrder": 4, + "type": "string", + "title": "API Key" + }, + "apiSecretKey": { + "propertyOrder": 5, + "type": "string", + "title": "API Secret Key" + }, + "stopWhenRateLimited": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "description": "Stop when hitting rate limit?", + "title": "Stop Upon Rate Limit" + }, + "searchQuery": { + "propertyOrder": 7, + "type": "string", + "description": "Up to 1024 characters (Limited by Twitter)", + "title": "Search Query", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "limit": { + "propertyOrder": 8, + "type": "integer", + "default": 100, + "description": "Maximum number of tweets to retrieve", + "title": "Limit" + } + }, + "required": [ + "apiKey", + "apiSecretKey", + "stopWhenRateLimited", + "searchQuery", + "limit" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TwitterSearch" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Twitter Search API", + "operatorDescription": "Retrieve data from Twitter Search API", + "operatorGroupName": "External API", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnPassiveAggressive", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnPassiveAggressive" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Passive Aggressive", + "operatorDescription": "Sklearn Passive Aggressive Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HTMLVisualizer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "htmlContentAttrName": { + "propertyOrder": 4, + "type": "string", + "title": "HTML content", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "htmlContentAttrName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HTMLVisualizer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "HTML Visualizer", + "operatorDescription": "Render the result of HTML content", + "operatorGroupName": "Media", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnComplementNaiveBayes", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnComplementNaiveBayes" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Complement Naive Bayes", + "operatorDescription": "Sklearn Complement Naive Bayes Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "URLFetcher", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "url": { + "propertyOrder": 4, + "type": "string", + "description": "Only accepts standard URL format", + "title": "URL" + }, + "decodingMethod": { + "propertyOrder": 5, + "type": "string", + "enum": [ + "UTF-8", + "RAW BYTES" + ], + "description": "The decoding method for the url content", + "title": "Decoding" + } + }, + "required": [ + "url", + "decodingMethod" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "URLFetcher" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "URL fetcher", + "operatorDescription": "Fetch the content of a single url", + "operatorGroupName": "External API", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "JavaUDF", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "import edu.uci.ics.texera.workflow.common.operators.map.MapOpExec;\nimport edu.uci.ics.amber.engine.common.model.tuple.Tuple;\nimport edu.uci.ics.amber.engine.common.model.tuple.TupleLike;\nimport scala.Function1;\nimport java.io.Serializable;\n\npublic class JavaUDFOpExec extends MapOpExec {\n public JavaUDFOpExec () {\n this.setMapFunc((Function1 & Serializable) this::processTuple);\n }\n \n public TupleLike processTuple(Tuple tuple) {\n return tuple;\n }\n}", + "description": "Input your code here", + "title": "Java UDF script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to lunch", + "title": "Worker count" + }, + "retainInputColumns": { + "propertyOrder": 6, + "type": "boolean", + "default": true, + "description": "Keep the original input columns?", + "title": "Retain input columns" + }, + "outputColumns": { + "propertyOrder": 7, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "Name of the newly added output columns that the UDF will produce, if any", + "title": "Extra output column(s)" + } + }, + "required": [ + "code", + "workers", + "retainInputColumns" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "JavaUDF" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Java UDF", + "operatorDescription": "User-defined function operator in Java script", + "operatorGroupName": "Java", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": true, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "supportReconfiguration": true, + "allowPortCustomization": true + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PieChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "value": { + "propertyOrder": 6, + "type": "string", + "description": "The value associated with slice of pie", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "name": { + "propertyOrder": 7, + "type": "string", + "description": "The name of the slice of pie", + "title": "Name Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "value", + "name" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PieChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Pie Chart", + "operatorDescription": "Visualize data in a Pie Chart", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "DictionaryMatcher", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Dictionary": { + "propertyOrder": 6, + "type": "string", + "description": "dictionary values separated by a comma", + "title": "Dictionary" + }, + "Attribute": { + "propertyOrder": 7, + "type": "string", + "description": "column name to match", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "result attribute": { + "propertyOrder": 8, + "type": "string", + "default": "matched", + "description": "column name of the matching result", + "title": "Result attribute" + }, + "Matching type": { + "propertyOrder": 9, + "type": "string", + "enum": [ + "Scan", + "Substring", + "Conjunction" + ], + "title": "Matching type" + } + }, + "required": [ + "Dictionary", + "Attribute", + "result attribute", + "Matching type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "DictionaryMatcher" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dictionary matcher", + "operatorDescription": "Matches tuples if they appear in a given dictionary", + "operatorGroupName": "Search", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "UnnestString", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Delimiter": { + "propertyOrder": 6, + "type": "string", + "default": ",", + "description": "string that separates the data", + "title": "Delimiter" + }, + "Attribute": { + "propertyOrder": 7, + "type": "string", + "description": "column of the string to unnest", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Result attribute": { + "propertyOrder": 8, + "type": "string", + "default": "unnestResult", + "description": "column name of the unnest result", + "title": "Result attribute" + } + }, + "required": [ + "Delimiter", + "Attribute", + "Result attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "UnnestString" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Unnest String", + "operatorDescription": "Unnest the string values in the column separated by a delimiter to multiple values", + "operatorGroupName": "Utilities", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "BubbleChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "xValue": { + "propertyOrder": 6, + "type": "string", + "description": "Data column for the x-axis", + "title": "X-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "yValue": { + "propertyOrder": 7, + "type": "string", + "description": "Data column for the y-axis", + "title": "Y-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "zValue": { + "propertyOrder": 8, + "type": "string", + "description": "Data column to determine bubble size", + "title": "Z-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "enableColor": { + "propertyOrder": 9, + "type": "boolean", + "default": false, + "description": "Colors bubbles using a data column", + "title": "Enable Color" + }, + "colorCategory": { + "propertyOrder": 10, + "type": "string", + "description": "Picks data column to color bubbles with if color is enabled", + "title": "Color-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "xValue", + "yValue", + "zValue", + "enableColor", + "colorCategory" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "BubbleChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Bubble Chart", + "operatorDescription": "a 3D Scatter Plot; Bubbles are graphed using x and y labels, and their sizes determined by a z-value.", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "RedditSearch", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "clientId": { + "propertyOrder": 4, + "type": "string", + "description": "Client id that uses to access Reddit API", + "title": "Client Id" + }, + "clientSecret": { + "propertyOrder": 5, + "type": "string", + "description": "Client secret that uses to access Reddit API", + "title": "Client Secret" + }, + "query": { + "propertyOrder": 6, + "type": "string", + "description": "Search query", + "title": "Query" + }, + "limit": { + "propertyOrder": 7, + "type": "integer", + "default": 100, + "description": "Up to 1000", + "title": "Limit" + }, + "sorting": { + "propertyOrder": 8, + "type": "string", + "default": "none", + "enum": [ + "none", + "controversial", + "gilded", + "hot", + "new", + "rising", + "top" + ], + "description": "The sorting method, hot, new, etc.", + "title": "Sorting" + } + }, + "required": [ + "clientId", + "clientSecret", + "query", + "limit", + "sorting" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "RedditSearch" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Reddit Search", + "operatorDescription": "Search for recent posts with python-wrapped Reddit API, PRAW", + "operatorGroupName": "External API", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SVRTrainer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "paraList": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/HyperParameters(SklearnAdvancedSVRParameters)" + }, + "title": "Parameter Setting" + }, + "groundTruthAttribute": { + "propertyOrder": 5, + "type": "string", + "description": "Ground truth attribute column", + "title": "Ground Truth Attribute Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Selected Features": { + "propertyOrder": 8, + "type": "array", + "items": { + "type": "string" + }, + "description": "Features used to train the model", + "title": "Selected Features", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "paraList", + "groundTruthAttribute", + "Selected Features" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SVRTrainer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HyperParameters(SklearnAdvancedSVRParameters)": { + "type": "object", + "additionalProperties": false, + "properties": { + "parameter": { + "propertyOrder": 1, + "type": "string", + "enum": [ + "C", + "kernel", + "gamma", + "degree", + "coef0", + "tol", + "probability", + "verbose", + "epsilon", + "cache_size", + "max_iter" + ], + "description": "Choose the name of the parameter", + "title": "Parameter" + }, + "parametersSource": { + "propertyOrder": 2, + "type": "boolean", + "default": false, + "description": "Parameter from workflow", + "title": "Workflow" + }, + "attribute": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 1 + }, + "value": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "true", + "hideOnNull": true + } + }, + "required": [ + "parameter", + "parametersSource" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "SVM Regressor", + "operatorDescription": "Sklearn SVM Regressor Operator", + "operatorGroupName": "Advanced Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "parameter", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "RUDF", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "# If using Table API:\n# function(table, port) { \n# return (table) \n# }\n\n# If using Tuple API:\n# library(coro)\n# coro::generator(function(tuple, port) {\n# yield (tuple)\n# })", + "description": "Input your code here", + "title": "R UDF Script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to lunch", + "title": "Worker count" + }, + "useTupleAPI": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "description": "Check this box to use Tuple API, leave unchecked to use Table API", + "title": "Use Tuple API?" + }, + "retainInputColumns": { + "propertyOrder": 7, + "type": "boolean", + "default": true, + "description": "Keep the original input columns?", + "title": "Retain input columns" + }, + "outputColumns": { + "propertyOrder": 8, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "Name of the newly added output columns that the UDF will produce, if any", + "title": "Extra output column(s)" + } + }, + "required": [ + "code", + "workers", + "useTupleAPI", + "retainInputColumns" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "RUDF" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "R UDF", + "operatorDescription": "User-defined function operator in R script", + "operatorGroupName": "R", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": true, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "BoxViolinPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "value": { + "propertyOrder": 1, + "type": "string", + "description": "Data column for box plot", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Quartile Method": { + "propertyOrder": 2, + "type": "string", + "default": "linear", + "enum": [ + "linear", + "inclusive", + "exclusive" + ], + "title": "Quartile Method" + }, + "horizontalOrientation": { + "propertyOrder": 3, + "type": "boolean", + "default": false, + "description": "Orientation style", + "title": "Horizontal Orientation" + }, + "violinPlot": { + "propertyOrder": 4, + "type": "boolean", + "default": false, + "description": "Check this box to overlay a violin plot on the box plot; otherwise, show only the box plot", + "title": "Violin Plot" + }, + "dummyPropertyList": { + "propertyOrder": 5, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [ + "value", + "Quartile Method", + "horizontalOrientation", + "violinPlot" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "BoxViolinPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Box/Violin Plot", + "operatorDescription": "Visualize data using either a Box Plot or a Violin Plot. Box plots are drawn as a box with a vertical line down the middle which is mean value, and has horizontal lines attached to each side (known as “whiskers”). Violin plots provide more detail by showing a smoothed density curve on each side, and also include a box plot inside for comparison.", + "operatorGroupName": "Statistical", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnAdaptiveBoosting", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnAdaptiveBoosting" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Adaptive Boosting", + "operatorDescription": "Sklearn Adaptive Boosting Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Scatterplot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "xColumn": { + "enum": [ + "integer", + "double" + ] + }, + "yColumn": { + "enum": [ + "integer", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "xColumn": { + "propertyOrder": 4, + "type": "string", + "description": "X Column", + "title": "X-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "yColumn": { + "propertyOrder": 5, + "type": "string", + "description": "Y Column", + "title": "Y-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "colorColumn": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Dots will be assigned different colors based on their values of this column", + "title": "Color-Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "xLogScale": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Values in X-column is log-scaled", + "title": "log scale X" + }, + "yLogScale": { + "propertyOrder": 8, + "type": "boolean", + "default": false, + "description": "Values in Y-column is log-scaled", + "title": "log scale Y" + }, + "hoverName": { + "propertyOrder": 9, + "nullable": true, + "type": "string", + "description": "Column value to display when a dot is hovered over", + "title": "Hover column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "xColumn", + "yColumn", + "xLogScale", + "yLogScale" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Scatterplot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Scatter Plot", + "operatorDescription": "View the result in a scatterplot", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnPerceptron", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnPerceptron" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Linear Perceptron", + "operatorDescription": "Sklearn Linear Perceptron Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "KeywordSearch", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "attribute": { + "propertyOrder": 4, + "type": "string", + "description": "column to search keyword on", + "title": "attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "keyword": { + "propertyOrder": 5, + "type": "string", + "description": "keywords", + "title": "keywords" + } + }, + "required": [ + "attribute", + "keyword" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "KeywordSearch" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Keyword Search", + "operatorDescription": "Search for keyword(s) in a string column", + "operatorGroupName": "Search", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PythonUDFV2", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "# Choose from the following templates:\n# \n# from pytexera import *\n# \n# class ProcessTupleOperator(UDFOperatorV2):\n# \n# @overrides\n# def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n# yield tuple_\n# \n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "description": "Input your code here", + "title": "Python script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to lunch", + "title": "Worker count" + }, + "retainInputColumns": { + "propertyOrder": 6, + "type": "boolean", + "default": true, + "description": "Keep the original input columns?", + "title": "Retain input columns" + }, + "outputColumns": { + "propertyOrder": 7, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "Name of the newly added output columns that the UDF will produce, if any", + "title": "Extra output column(s)" + } + }, + "required": [ + "code", + "workers", + "retainInputColumns" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PythonUDFV2" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Python UDF", + "operatorDescription": "User-defined function operator in Python script", + "operatorGroupName": "Python", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": true, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "supportReconfiguration": true, + "allowPortCustomization": true + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnLogisticRegression", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnLogisticRegression" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Logistic Regression", + "operatorDescription": "Sklearn Logistic Regression Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnRandomForest", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnRandomForest" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Random Forest", + "operatorDescription": "Sklearn Random Forest Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TypeCasting", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "typeCastingUnits": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/TypeCastingUnit" + }, + "description": "Multiple type castings", + "title": "TypeCasting Units" + } + }, + "required": [ + "typeCastingUnits" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TypeCasting" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "TypeCastingUnit": { + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "attribute": { + "allOf": [ + { + "if": { + "resultType": { + "valEnum": [ + "integer" + ] + } + }, + "then": { + "enum": [ + "string", + "long", + "double", + "boolean" + ] + } + }, + { + "if": { + "resultType": { + "valEnum": [ + "double" + ] + } + }, + "then": { + "enum": [ + "string", + "integer", + "long", + "boolean" + ] + } + }, + { + "if": { + "resultType": { + "valEnum": [ + "boolean" + ] + } + }, + "then": { + "enum": [ + "string", + "integer", + "long", + "double" + ] + } + }, + { + "if": { + "resultType": { + "valEnum": [ + "long" + ] + } + }, + "then": { + "enum": [ + "string", + "integer", + "double", + "boolean", + "timestamp" + ] + } + }, + { + "if": { + "resultType": { + "valEnum": [ + "timestamp" + ] + } + }, + "then": { + "enum": [ + "string", + "long" + ] + } + } + ] + } + }, + "properties": { + "attribute": { + "propertyOrder": 1, + "type": "string", + "description": "Attribute for type casting", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "resultType": { + "propertyOrder": 2, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "description": "Result type after type casting", + "title": "Cast type" + } + }, + "required": [ + "attribute", + "resultType" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Type Casting", + "operatorDescription": "Cast between types", + "operatorGroupName": "Data Cleaning", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnGaussianNaiveBayes", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnGaussianNaiveBayes" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Gaussian Naive Bayes", + "operatorDescription": "Sklearn Gaussian Naive Bayes Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "AsterixDBSource", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "host": { + "propertyOrder": 4, + "type": "string", + "title": "Host", + "enable-presets": true + }, + "port": { + "propertyOrder": 5, + "type": "string", + "default": "default", + "description": "A port number or 'default'", + "title": "Port", + "enable-presets": true + }, + "database": { + "propertyOrder": 6, + "type": "string", + "title": "Database", + "enable-presets": true + }, + "table": { + "propertyOrder": 7, + "type": "string", + "title": "Table Name", + "enable-presets": true + }, + "limit": { + "propertyOrder": 8, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 9, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "keywordSearch": { + "propertyOrder": 10, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Keyword Search?", + "toggleHidden": [ + "keywordSearchByColumn", + "keywords" + ] + }, + "keywordSearchByColumn": { + "propertyOrder": 11, + "nullable": true, + "type": "string", + "title": "Keyword Search Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "keywords": { + "propertyOrder": 12, + "nullable": true, + "type": "string", + "description": "\"['hello', 'world'], {'mode':'any'}\" OR \"['hello', 'world'], {'mode':'all'}\"", + "title": "Keywords to Search", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "progressive": { + "propertyOrder": 13, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Progressive?", + "toggleHidden": [ + "batchByColumn", + "min", + "max", + "interval" + ] + }, + "batchByColumn": { + "propertyOrder": 14, + "nullable": true, + "type": "string", + "title": "Batch by Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "min": { + "propertyOrder": 15, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Min", + "dependOn": "batchByColumn" + }, + "max": { + "propertyOrder": 16, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Max", + "dependOn": "batchByColumn" + }, + "interval": { + "propertyOrder": 17, + "type": "integer", + "default": 1000000000, + "title": "Batch by Interval", + "dependOn": "batchByColumn" + }, + "geoSearch": { + "propertyOrder": 18, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Geo Search?", + "toggleHidden": [ + "geoSearchByColumns", + "geoSearchBoundingBox" + ] + }, + "geoSearchByColumns": { + "propertyOrder": 19, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "description": "column(s) to check if any of them is in the bounding box below", + "title": "Geo Search By Columns", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + }, + "geoSearchBoundingBox": { + "propertyOrder": 20, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "description": "at least 2 entries should be provided to form a bounding box. format of each entry: long, lat", + "title": "Geo Search Bounding Box" + }, + "regexSearch": { + "propertyOrder": 21, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Regex Search?", + "toggleHidden": [ + "regexSearchByColumn", + "regex" + ] + }, + "regexSearchByColumn": { + "propertyOrder": 22, + "nullable": true, + "type": "string", + "title": "Regex Search By Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "regex": { + "propertyOrder": 23, + "nullable": true, + "type": "string", + "title": "Regex to Search", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "filterCondition": { + "propertyOrder": 24, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Filter Condition?", + "toggleHidden": [ + "predicates" + ] + }, + "predicates": { + "propertyOrder": 27, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/FilterPredicate" + }, + "description": "multiple predicates in OR", + "title": "Predicates" + } + }, + "required": [ + "host", + "port", + "database", + "table", + "interval" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "AsterixDBSource" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "FilterPredicate": { + "type": "object", + "additionalProperties": false, + "properties": { + "attribute": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "condition": { + "propertyOrder": 2, + "type": "string", + "enum": [ + "=", + ">", + ">=", + "<", + "<=", + "!=", + "is null", + "is not null" + ], + "title": "Condition" + }, + "value": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "condition", + "hideType": "regex", + "hideExpectedValue": "is null|is not null" + } + }, + "required": [ + "attribute", + "condition" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "AsterixDB Source", + "operatorDescription": "Read data from a AsterixDB instance", + "operatorGroupName": "Database Connector", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "DualInputPortsPythonUDFV2", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "code": { + "propertyOrder": 4, + "type": "string", + "default": "# Choose from the following templates:\n# \n# from pytexera import *\n# \n# class ProcessTupleOperator(UDFOperatorV2):\n# \n# @overrides\n# def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n# yield tuple_\n# \n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "description": "Input your code here", + "title": "Python script" + }, + "workers": { + "propertyOrder": 5, + "type": "integer", + "default": 1, + "description": "Specify how many parallel workers to lunch", + "title": "Worker count" + }, + "retainInputColumns": { + "propertyOrder": 6, + "type": "boolean", + "default": true, + "description": "Keep the original input columns?", + "title": "Retain input columns" + }, + "outputColumns": { + "propertyOrder": 7, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Attribute" + }, + "description": "Name of the newly added output columns that the UDF will produce, if any", + "title": "Extra output column(s)" + } + }, + "required": [ + "code", + "workers", + "retainInputColumns" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "DualInputPortsPythonUDFV2" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "Attribute": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Attribute Name" + }, + "attributeType": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + } + } + } + } + }, + "additionalMetadata": { + "userFriendlyName": "2-in Python UDF", + "operatorDescription": "User-defined function operator in Python script", + "operatorGroupName": "Python", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "model", + "allowMultiLinks": true, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "tuples", + "allowMultiLinks": true, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Histogram", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "color": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "description": "Column for differentiating data by its value.", + "title": "Color Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "separateBy": { + "propertyOrder": 5, + "nullable": true, + "type": "string", + "description": "Column for separating histogram chart by its value.", + "title": "SeparateBy Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "marginal": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Distribution type (rug, box, violin).", + "title": "Distribution Type" + }, + "pattern": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "description": "Add texture to the chart based on an attribute", + "title": "Pattern", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "value": { + "propertyOrder": 10, + "type": "string", + "description": "Column for counting values.", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "value" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Histogram" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Histogram", + "operatorDescription": "Visualize data in a Histogram Chart", + "operatorGroupName": "Statistical", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnDummyClassifier", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnDummyClassifier" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dummy Classifier", + "operatorDescription": "Sklearn Dummy Classifier Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Distinct", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Distinct" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Distinct", + "operatorDescription": "Remove duplicate tuples", + "operatorGroupName": "Data Cleaning", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "NetworkGraph", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "source": { + "propertyOrder": 4, + "type": "string", + "description": "Source node for edge in graph", + "title": "Source Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "destination": { + "propertyOrder": 5, + "type": "string", + "description": "Destination node for edge in graph", + "title": "Destination Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "title": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "default": "Network Graph", + "title": "Title" + } + }, + "required": [ + "source", + "destination" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "NetworkGraph" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Network Graph", + "operatorDescription": "Visualize data in a network graph", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "WaterfallChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "xColumn": { + "propertyOrder": 6, + "type": "string", + "description": "The column representing categories or stages", + "title": "X Axis Values", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "yColumn": { + "propertyOrder": 7, + "type": "string", + "description": "The column representing numeric values for each stage", + "title": "Y Axis Values", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "xColumn", + "yColumn" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "WaterfallChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Waterfall Chart", + "operatorDescription": "Visualize data as a waterfall chart", + "operatorGroupName": "Financial", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Limit", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "limit": { + "propertyOrder": 4, + "type": "integer", + "description": "the max number of output rows", + "title": "Limit" + } + }, + "required": [ + "limit" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Limit" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Limit", + "operatorDescription": "Limit the number of output rows", + "operatorGroupName": "Data Cleaning", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Scorer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "isRegression": { + "propertyOrder": 4, + "type": "boolean", + "default": false, + "description": "Choose to solve a regression task", + "title": "Regression" + }, + "actualValueColumn": { + "propertyOrder": 5, + "type": "string", + "description": "Specify the label attribute", + "title": "Actual Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "predictValueColumn": { + "propertyOrder": 6, + "type": "string", + "description": "Specify the attribute generated by the model", + "title": "Predicted Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "classificationFlag": { + "propertyOrder": 9, + "nullable": true, + "type": "array", + "items": { + "type": "string", + "enum": [ + "Accuracy", + "Precision Score", + "Recall Score", + "F1 Score" + ] + }, + "description": "Select classification tasks metrics", + "title": "Scorer Functions", + "hideTarget": "isRegression", + "hideType": "equals", + "hideExpectedValue": "true" + }, + "regressionFlag": { + "propertyOrder": 10, + "nullable": true, + "type": "array", + "items": { + "type": "string", + "enum": [ + "MSE", + "RMSE", + "MAE", + "R2" + ] + }, + "description": "Select regression tasks metrics", + "title": "Scorer Functions", + "hideTarget": "isRegression", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "isRegression", + "actualValueColumn", + "predictValueColumn" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Scorer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Machine Learning Scorer", + "operatorDescription": "Scorer for machine learning models", + "operatorGroupName": "Machine Learning General", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnExtraTrees", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnExtraTrees" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Extra Trees", + "operatorDescription": "Sklearn Extra Trees Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "FileScan", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fileName": { + "propertyOrder": 4, + "type": "string", + "title": "File" + }, + "encoding": { + "propertyOrder": 5, + "type": "string", + "default": "UTF_8", + "enum": [ + "UTF_8", + "UTF_16", + "US_ASCII" + ], + "title": "Encoding", + "hideTarget": "attributeType", + "hideType": "equals", + "hideExpectedValue": "binary" + }, + "extract": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "title": "Extract" + }, + "outputFileName": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "title": "Include Filename", + "hideTarget": "extract", + "hideType": "equals", + "hideExpectedValue": "false" + }, + "attributeType": { + "propertyOrder": 8, + "type": "string", + "default": "string", + "enum": [ + "string", + "single string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + }, + "attributeName": { + "propertyOrder": 9, + "type": "string", + "default": "line", + "title": "Attribute Name" + }, + "fileScanLimit": { + "propertyOrder": 10, + "nullable": true, + "type": "integer", + "title": "Limit", + "hideTarget": "attributeType", + "hideType": "regex", + "hideExpectedValue": "^binary$|^single string$" + }, + "fileScanOffset": { + "propertyOrder": 11, + "nullable": true, + "type": "integer", + "title": "Offset", + "hideTarget": "attributeType", + "hideType": "regex", + "hideExpectedValue": "^binary$|^single string$" + } + }, + "required": [ + "fileName", + "encoding", + "extract", + "outputFileName", + "attributeType", + "attributeName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "FileScan" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": " File Scan", + "operatorDescription": "Scan data from a file", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "GanttChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "start": { + "enum": [ + "timestamp" + ] + }, + "finish": { + "enum": [ + "timestamp" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "pattern": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "description": "Add texture to the chart based on an attribute", + "title": "Pattern", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "start": { + "propertyOrder": 7, + "type": "string", + "description": "the start timestamp of the task", + "title": "Start Datetime Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "finish": { + "propertyOrder": 8, + "type": "string", + "description": "the end timestamp of the task", + "title": "Finish Datetime Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "task": { + "propertyOrder": 9, + "type": "string", + "description": "the name of the task", + "title": "Task Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "color": { + "propertyOrder": 10, + "nullable": true, + "type": "string", + "description": "column to color tasks", + "title": "Color Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "start", + "finish", + "task" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "GanttChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Gantt Chart", + "operatorDescription": "A Gantt chart is a type of bar chart that illustrates a project schedule. The chart lists the tasks to be performed on the vertical axis, and time intervals on the horizontal axis. The width of the horizontal bars in the graph shows the duration of each activity.", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TernaryPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "firstVariable": { + "propertyOrder": 6, + "type": "string", + "description": "First variable data field", + "title": "Variable 1", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "secondVariable": { + "propertyOrder": 7, + "type": "string", + "description": "Second variable data field", + "title": "Variable 2", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "thirdVariable": { + "propertyOrder": 8, + "type": "string", + "description": "Third variable data field", + "title": "Variable 3", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "colorEnabled": { + "propertyOrder": 9, + "type": "boolean", + "default": false, + "description": "Optionally color points using a data field", + "title": "Categorize by Color" + }, + "colorDataField": { + "propertyOrder": 10, + "nullable": true, + "type": "string", + "description": "Specify the data field to color", + "title": "Color Data Field", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "firstVariable", + "secondVariable", + "thirdVariable", + "colorEnabled" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TernaryPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Ternary Plot", + "operatorDescription": "Points are graphed on a Ternary Plot using 3 specified data fields", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SVCTrainer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "paraList": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/HyperParameters(SklearnAdvancedSVCParameters)" + }, + "title": "Parameter Setting" + }, + "groundTruthAttribute": { + "propertyOrder": 5, + "type": "string", + "description": "Ground truth attribute column", + "title": "Ground Truth Attribute Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Selected Features": { + "propertyOrder": 8, + "type": "array", + "items": { + "type": "string" + }, + "description": "Features used to train the model", + "title": "Selected Features", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "paraList", + "groundTruthAttribute", + "Selected Features" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SVCTrainer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HyperParameters(SklearnAdvancedSVCParameters)": { + "type": "object", + "additionalProperties": false, + "properties": { + "parameter": { + "propertyOrder": 1, + "type": "string", + "enum": [ + "C", + "kernel", + "gamma", + "degree", + "coef0", + "tol", + "probability" + ], + "description": "Choose the name of the parameter", + "title": "Parameter" + }, + "parametersSource": { + "propertyOrder": 2, + "type": "boolean", + "default": false, + "description": "Parameter from workflow", + "title": "Workflow" + }, + "attribute": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 1 + }, + "value": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "true", + "hideOnNull": true + } + }, + "required": [ + "parameter", + "parametersSource" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "SVM Classifier", + "operatorDescription": "Sklearn SVM Classifier Operator", + "operatorGroupName": "Advanced Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "parameter", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnLinearRegression", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "degree": { + "propertyOrder": 5, + "type": "integer", + "description": "Degree of polynomial function", + "title": "Degree" + } + }, + "required": [ + "target", + "degree" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnLinearRegression" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Linear Regression", + "operatorDescription": "Sklearn Linear Regression Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "MySQLSource", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "host": { + "propertyOrder": 4, + "type": "string", + "title": "Host", + "enable-presets": true + }, + "port": { + "propertyOrder": 5, + "type": "string", + "default": "default", + "description": "A port number or 'default'", + "title": "Port", + "enable-presets": true + }, + "database": { + "propertyOrder": 6, + "type": "string", + "title": "Database", + "enable-presets": true + }, + "table": { + "propertyOrder": 7, + "type": "string", + "title": "Table Name", + "enable-presets": true + }, + "username": { + "propertyOrder": 8, + "type": "string", + "title": "Username", + "enable-presets": true + }, + "password": { + "propertyOrder": 9, + "type": "string", + "title": "Password", + "widget": { + "formlyConfig": { + "templateOptions": { + "type": "password" + } + } + } + }, + "limit": { + "propertyOrder": 10, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 11, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "keywordSearch": { + "propertyOrder": 12, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Keyword Search?", + "toggleHidden": [ + "keywordSearchByColumn", + "keywords" + ] + }, + "keywordSearchByColumn": { + "propertyOrder": 13, + "nullable": true, + "type": "string", + "title": "Keyword Search Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "keywords": { + "propertyOrder": 14, + "nullable": true, + "type": "string", + "title": "Keywords to Search", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "progressive": { + "propertyOrder": 15, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Progressive?", + "toggleHidden": [ + "batchByColumn", + "min", + "max", + "interval" + ] + }, + "batchByColumn": { + "propertyOrder": 16, + "nullable": true, + "type": "string", + "title": "Batch by Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "min": { + "propertyOrder": 17, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Min", + "dependOn": "batchByColumn" + }, + "max": { + "propertyOrder": 18, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Max", + "dependOn": "batchByColumn" + }, + "interval": { + "propertyOrder": 19, + "type": "integer", + "default": 1000000000, + "title": "Batch by Interval", + "dependOn": "batchByColumn" + } + }, + "required": [ + "host", + "port", + "database", + "table", + "username", + "password", + "interval" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "MySQLSource" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "MySQL Source", + "operatorDescription": "Read data from a MySQL instance", + "operatorGroupName": "Database Connector", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "CSVOldFileScan", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fileName": { + "propertyOrder": 4, + "type": "string", + "title": "File" + }, + "fileEncoding": { + "propertyOrder": 5, + "type": "string", + "default": "UTF_8", + "enum": [ + "UTF_8", + "UTF_16", + "US_ASCII" + ], + "description": "decoding charset to use on input", + "title": "File Encoding" + }, + "limit": { + "propertyOrder": 6, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 7, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "customDelimiter": { + "propertyOrder": 8, + "nullable": true, + "type": "string", + "default": ",", + "description": "delimiter to separate each line into fields", + "title": "Delimiter" + }, + "hasHeader": { + "propertyOrder": 9, + "type": "boolean", + "default": true, + "description": "whether the CSV file contains a header line", + "title": "Header" + } + }, + "required": [ + "fileName", + "fileEncoding", + "hasHeader" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "CSVOldFileScan" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "CSVOld File Scan", + "operatorDescription": "Scan data from a CSVOld file", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "CSVFileScan", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fileName": { + "propertyOrder": 4, + "type": "string", + "title": "File" + }, + "fileEncoding": { + "propertyOrder": 5, + "type": "string", + "default": "UTF_8", + "enum": [ + "UTF_8", + "UTF_16", + "US_ASCII" + ], + "description": "decoding charset to use on input", + "title": "File Encoding" + }, + "limit": { + "propertyOrder": 6, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 7, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "customDelimiter": { + "propertyOrder": 8, + "nullable": true, + "type": "string", + "default": ",", + "description": "delimiter to separate each line into fields", + "title": "Delimiter" + }, + "hasHeader": { + "propertyOrder": 9, + "type": "boolean", + "default": true, + "description": "whether the CSV file contains a header line", + "title": "Header" + } + }, + "required": [ + "fileName", + "fileEncoding", + "hasHeader" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "CSVFileScan" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "CSV File Scan", + "operatorDescription": "Scan data from a CSV file", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "FunnelPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "title": "string" + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "x": { + "propertyOrder": 4, + "type": "string", + "description": "Data column for the x-axis", + "title": "X Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 5, + "type": "string", + "description": "Data column for the y-axis", + "title": "Y Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "color": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Column to categorically colorize funnel sections", + "title": "Color Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "x", + "y" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "FunnelPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Funnel Plot", + "operatorDescription": "Visualize data in a Funnel Plot", + "operatorGroupName": "Financial", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Projection", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "isDrop": { + "propertyOrder": 4, + "type": "boolean", + "default": false, + "description": "check to drop the selected attributes", + "title": "Drop Option" + }, + "attributes": { + "propertyOrder": 5, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/AttributeUnit" + }, + "title": "Attributes" + } + }, + "required": [ + "isDrop" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Projection" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "AttributeUnit": { + "type": "object", + "additionalProperties": false, + "properties": { + "originalAttribute": { + "propertyOrder": 1, + "type": "string", + "description": "Attribute name in the schema", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "alias": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "description": "Renamed attribute name", + "title": "Alias" + } + }, + "required": [ + "originalAttribute" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Projection", + "operatorDescription": "Keeps or drops the column", + "operatorGroupName": "Data Cleaning", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Filter", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "predicates": { + "propertyOrder": 6, + "type": "array", + "items": { + "$ref": "#/definitions/FilterPredicate" + }, + "description": "multiple predicates in OR", + "title": "Predicates" + } + }, + "required": [ + "predicates" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Filter" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "FilterPredicate": { + "type": "object", + "additionalProperties": false, + "properties": { + "attribute": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "condition": { + "propertyOrder": 2, + "type": "string", + "enum": [ + "=", + ">", + ">=", + "<", + "<=", + "!=", + "is null", + "is not null" + ], + "title": "Condition" + }, + "value": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "condition", + "hideType": "regex", + "hideExpectedValue": "is null|is not null" + } + }, + "required": [ + "attribute", + "condition" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Filter", + "operatorDescription": "Performs a filter operation", + "operatorGroupName": "Data Cleaning", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnRidge", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnRidge" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Ridge Regression", + "operatorDescription": "Sklearn Ridge Regression Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Intersect", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Intersect" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Intersect", + "operatorDescription": "Take the intersect of two inputs", + "operatorGroupName": "Set", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnPrediction", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Model Attribute": { + "propertyOrder": 6, + "type": "string", + "default": "model", + "description": "attribute corresponding to ML model", + "title": "Model Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Output Attribute Name": { + "propertyOrder": 7, + "type": "string", + "default": "prediction", + "description": "attribute name of the prediction result", + "title": "Output Attribute Name" + }, + "Ground Truth Attribute Name to Ignore": { + "propertyOrder": 8, + "nullable": true, + "type": "string", + "description": "attribute name of the ground truth", + "title": "Ground Truth Attribute Name to Ignore", + "autofill": "attributeName", + "autofillAttributeOnPort": 1 + } + }, + "required": [ + "Model Attribute", + "Output Attribute Name" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnPrediction" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Sklearn Prediction", + "operatorDescription": "Skleanr Prediction Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "model", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SymmetricDifference", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SymmetricDifference" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "SymmetricDifference", + "operatorDescription": "find the symmetric difference (the set of elements which are in either of the sets, but not in their intersection) of two inputs", + "operatorGroupName": "Set", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "FigureFactoryTable", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fontSize": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "description": "Font size of the Figure Factory Table", + "title": "Font Size" + }, + "fontColor": { + "propertyOrder": 5, + "nullable": true, + "type": "string", + "description": "Font color of the Figure Factory Table", + "title": "Font Color (Hex Code)" + }, + "rowHeight": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Row height of the Figure Factory Table", + "title": "Row Height" + }, + "add attribute": { + "propertyOrder": 9, + "type": "array", + "items": { + "$ref": "#/definitions/FigureFactoryTableConfig" + }, + "description": "List of columns to include in the figure factory table", + "title": "Add attribute" + } + }, + "required": [ + "add attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "FigureFactoryTable" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "FigureFactoryTableConfig": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "attributeName" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Figure Factory Table", + "operatorDescription": "Visualize data in a figure factory table", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "FilledAreaPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "x": { + "propertyOrder": 4, + "type": "string", + "description": "The attribute for your x-axis", + "title": "X-axis Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 5, + "type": "string", + "description": "The attribute for your y-axis", + "title": "Y-axis Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "lineGroup": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "The attribute for group of each line", + "title": "Line Group", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "color": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "description": "Choose an attribute to color the plot", + "title": "Color", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "facetColumn": { + "propertyOrder": 8, + "type": "boolean", + "description": "Do you want to split the graph", + "title": "Split Plot by Line Group" + }, + "pattern": { + "propertyOrder": 9, + "nullable": true, + "type": "string", + "description": "Add texture to the chart based on an attribute", + "title": "Pattern", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "x", + "y", + "facetColumn" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "FilledAreaPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Filled Area Plot", + "operatorDescription": "Visualize data in filled area plot", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnRidgeCV", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnRidgeCV" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Ridge Regression Cross Validation", + "operatorDescription": "Sklearn Ridge Regression Cross Validation Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "IcicleChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "hierarchy": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/HierarchySection" + }, + "description": "hierarchy of attributes from a root (higher-level category) to leaves (lower-level category)", + "title": "Hierarchy Path" + }, + "value": { + "propertyOrder": 7, + "type": "string", + "description": "the value associated with the size of each sector in the chart", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "hierarchy", + "value" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "IcicleChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HierarchySection": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "attributeName" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Icicle Chart", + "operatorDescription": "Visualize hierarchical data from root to leaves", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Regex", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "caseInsensitive": { + "propertyOrder": 4, + "type": "boolean", + "default": false, + "description": "regex match is case sensitive", + "title": "Case Insensitive" + }, + "attribute": { + "propertyOrder": 7, + "type": "string", + "description": "column to search regex on", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "regex": { + "propertyOrder": 8, + "type": "string", + "description": "regular expression", + "title": "Regex" + } + }, + "required": [ + "caseInsensitive", + "attribute", + "regex" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Regex" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Regular Expression", + "operatorDescription": "Search a regular expression in a string column", + "operatorGroupName": "Search", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HeatMap", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "x": { + "propertyOrder": 6, + "type": "string", + "description": "the values along the x-axis", + "title": "Value X Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 7, + "type": "string", + "description": "the values along the y-axis", + "title": "Value Y Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Values": { + "propertyOrder": 8, + "type": "string", + "description": "the values of the heatmap", + "title": "Values", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "x", + "y", + "Values" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HeatMap" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Heatmap", + "operatorDescription": "Visualize data in a HeatMap Chart", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TablesPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "add attribute": { + "propertyOrder": 6, + "type": "array", + "items": { + "$ref": "#/definitions/TablesConfig" + }, + "description": "List of columns to include in the table chart", + "title": "Add attribute" + } + }, + "required": [ + "add attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TablesPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "TablesConfig": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "attributeName" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Tables Plot", + "operatorDescription": "Visualize data in a table chart.", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HierarchyChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "hierarchyChartType": { + "propertyOrder": 4, + "type": "string", + "enum": [ + "treemap", + "sunburst" + ], + "description": "Treemap or Sunburst", + "title": "Chart Type" + }, + "hierarchy": { + "propertyOrder": 5, + "type": "array", + "items": { + "$ref": "#/definitions/HierarchySection" + }, + "description": "Hierarchy of attributes from a higher-level category to lower-level category", + "title": "Hierarchy Path" + }, + "value": { + "propertyOrder": 8, + "type": "string", + "description": "The value associated with the size of each sector in the chart", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "hierarchyChartType", + "hierarchy", + "value" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HierarchyChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HierarchySection": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "attributeName" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hierarchy Chart", + "operatorDescription": "Visualize data in hierarchy", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnExtraTree", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnExtraTree" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Extra Tree", + "operatorDescription": "Sklearn Extra Tree Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Sort", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "attributes": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/SortCriteriaUnit" + }, + "description": "column to perform sorting on", + "title": "Attributes" + } + }, + "required": [ + "attributes" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Sort" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "SortCriteriaUnit": { + "type": "object", + "additionalProperties": false, + "properties": { + "attribute": { + "propertyOrder": 1, + "type": "string", + "description": "Attribute name to sort by", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "sortPreference": { + "propertyOrder": 2, + "type": "string", + "enum": [ + "ASC", + "DESC" + ], + "description": "Sort preference (ASC or DESC)", + "title": "Sort Preference" + } + }, + "required": [ + "attribute", + "sortPreference" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Sort", + "operatorDescription": "Sort based on the columns and sorting methods", + "operatorGroupName": "Sort", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Scatter3DChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "title": "string" + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "x": { + "propertyOrder": 6, + "type": "string", + "description": "Data column for the x-axis", + "title": "X Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 7, + "type": "string", + "description": "Data column for the y-axis", + "title": "Y Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "z": { + "propertyOrder": 8, + "type": "string", + "description": "Data column for the z-axis", + "title": "Z Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "x", + "y", + "z" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Scatter3DChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Scatter3D Chart", + "operatorDescription": "Visualize data in a Scatter3D Plot", + "operatorGroupName": "Advanced", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnBagging", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnBagging" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Bagging", + "operatorDescription": "Sklearn Bagging Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Difference", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Difference" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Difference", + "operatorDescription": "find the set difference of two inputs", + "operatorGroupName": "Set", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "left", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "right", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ContourPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "gridSize": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "default": "10", + "description": "Grid resolution of the final image", + "title": "Grid Size" + }, + "connectGaps": { + "propertyOrder": 5, + "type": "boolean", + "default": true, + "description": "Automatically fill in the missing parts", + "title": "Connect Gaps" + }, + "x": { + "propertyOrder": 8, + "type": "string", + "description": "The column name of X-axis", + "title": "x", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 9, + "type": "string", + "description": "The column name of Y-axis", + "title": "y", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "z": { + "propertyOrder": 10, + "type": "string", + "description": "The column name of color bar", + "title": "z", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Coloring Method": { + "propertyOrder": 11, + "nullable": true, + "type": "string", + "default": "heatmap", + "enum": [ + "heatmap", + "lines", + "none" + ], + "title": "Coloring Method" + } + }, + "required": [ + "connectGaps", + "x", + "y", + "z" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ContourPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Contour Plot", + "operatorDescription": "Displays terrain or gradient variations in a Contour Plot", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PythonLambdaFunction", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "lambdaAttributeUnits": { + "propertyOrder": 4, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/LambdaAttributeUnit" + }, + "title": "Add/Modify column(s)" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PythonLambdaFunction" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "LambdaAttributeUnit": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "hideTarget": "attributeName", + "hideType": "regex", + "hideExpectedValue": "Add New Column", + "additionalEnumValue": "Add New Column", + "autofillAttributeOnPort": 0 + }, + "newAttributeName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "New Attribute Name", + "hideTarget": "attributeName", + "hideType": "regex", + "hideExpectedValue": "(?!Add New Column).*", + "hideOnNull": true + }, + "attributeType": { + "propertyOrder": 3, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + }, + "expression": { + "propertyOrder": 4, + "type": "string", + "title": "Expression" + } + }, + "required": [ + "attributeName", + "attributeType", + "expression" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Python Lambda Function", + "operatorDescription": "Modify or add a new column with more ease", + "operatorGroupName": "Python", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "WordCloud", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "textColumn": { + "propertyOrder": 4, + "type": "string", + "title": "Text column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "topN": { + "propertyOrder": 5, + "nullable": true, + "type": "integer", + "default": 100, + "title": "Number of most frequent words", + "exclusiveMinimum": 0 + } + }, + "required": [ + "textColumn" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "WordCloud" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Word Cloud", + "operatorDescription": "Generate word cloud for texts", + "operatorGroupName": "Media", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "LineChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "yLabel": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "default": "Y Axis", + "description": "the label for y axis", + "title": "Y Label" + }, + "xLabel": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "default": "X Axis", + "description": "the label for x axis", + "title": "X Label" + }, + "lines": { + "propertyOrder": 8, + "type": "array", + "items": { + "$ref": "#/definitions/LineConfig" + }, + "title": "Lines" + } + }, + "required": [ + "lines" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "LineChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "LineConfig": { + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "yValue": { + "enum": [ + "integer", + "long", + "double" + ] + }, + "xValue": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "y": { + "propertyOrder": 1, + "type": "string", + "description": "value for y axis", + "title": "Y Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "x": { + "propertyOrder": 2, + "type": "string", + "description": "value for x axis", + "title": "X Value", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "mode": { + "propertyOrder": 3, + "type": "string", + "default": "line with dots", + "enum": [ + "line", + "dots", + "line with dots" + ], + "title": "Line Mode" + }, + "name": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "title": "Line Name" + }, + "color": { + "propertyOrder": 5, + "nullable": true, + "type": "string", + "description": "must be a valid CSS color or hex color string", + "title": "Line Color" + } + }, + "required": [ + "y", + "x", + "mode" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Line Chart", + "operatorDescription": "View the result in line chart", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "RandomKSampling", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "random k sample percentage": { + "propertyOrder": 6, + "type": "integer", + "description": "random k sampling with given percentage", + "title": "Random k sample percentage" + } + }, + "required": [ + "random k sample percentage" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "RandomKSampling" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Random K Sampling", + "operatorDescription": "random sampling with given percentage", + "operatorGroupName": "Utilities", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": true, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Split", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "k": { + "propertyOrder": 4, + "type": "integer", + "default": 80, + "description": "percentage of data going to the upper port", + "title": "Split Percentage" + }, + "random": { + "propertyOrder": 5, + "type": "boolean", + "default": true, + "description": "Shuffle the data based on a random seed", + "title": "Auto-Generate Seed" + }, + "seed": { + "propertyOrder": 6, + "type": "integer", + "default": 1, + "description": "An int for reproducible output across multiple run", + "title": "Seed", + "hideTarget": "random", + "hideType": "equals", + "hideExpectedValue": "true" + } + }, + "required": [ + "k", + "random", + "seed" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Split" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Split", + "operatorDescription": "Split data to two different ports", + "operatorGroupName": "Utilities", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": true, + "dynamicOutputPorts": true, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnMultiLayerPerceptron", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnMultiLayerPerceptron" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Multi-layer Perceptron", + "operatorDescription": "Sklearn Multi-layer Perceptron Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "BarChart", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fields": { + "propertyOrder": 4, + "type": "string", + "description": "Visualize categorical data in a Bar Chart", + "title": "Fields", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "categoryColumn": { + "propertyOrder": 5, + "nullable": true, + "type": "string", + "default": "No Selection", + "description": "Optional - Select a column to Color Code the Categories", + "title": "Category Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "horizontalOrientation": { + "propertyOrder": 6, + "type": "boolean", + "default": false, + "description": "Orientation Style", + "title": "Horizontal Orientation" + }, + "pattern": { + "propertyOrder": 7, + "nullable": true, + "type": "string", + "description": "Add texture to the chart based on an attribute", + "title": "Pattern", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "value": { + "propertyOrder": 10, + "type": "string", + "description": "The value associated with each category", + "title": "Value Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "fields", + "horizontalOrientation", + "value" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "BarChart" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Bar Chart", + "operatorDescription": "Visualize data in a Bar Chart", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HashJoin", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "buildAttributeName": { + "const": { + "$data": "probeAttributeName" + } + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "buildAttributeName": { + "propertyOrder": 4, + "type": "string", + "description": "attribute to be joined on the Left Input", + "title": "Left Input Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "probeAttributeName": { + "propertyOrder": 5, + "type": "string", + "description": "attribute to be joined on the Right Input", + "title": "Right Input Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 1 + }, + "joinType": { + "propertyOrder": 6, + "type": "string", + "default": "inner", + "enum": [ + "inner", + "left outer", + "right outer", + "full outer" + ], + "description": "select the join type to execute", + "title": "Join Type" + } + }, + "required": [ + "buildAttributeName", + "probeAttributeName", + "joinType" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HashJoin" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hash Join", + "operatorDescription": "join two inputs", + "operatorGroupName": "Join", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "left", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "right", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PythonTableReducer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "lambdaAttributeUnits": { + "propertyOrder": 4, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/LambdaAttributeUnit" + }, + "title": "Output columns" + } + }, + "required": [], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PythonTableReducer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "LambdaAttributeUnit": { + "type": "object", + "additionalProperties": false, + "properties": { + "attributeName": { + "propertyOrder": 1, + "type": "string", + "title": "Attribute Name", + "autofill": "attributeName", + "hideTarget": "attributeName", + "hideType": "regex", + "hideExpectedValue": "Add New Column", + "additionalEnumValue": "Add New Column", + "autofillAttributeOnPort": 0 + }, + "newAttributeName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "New Attribute Name", + "hideTarget": "attributeName", + "hideType": "regex", + "hideExpectedValue": "(?!Add New Column).*", + "hideOnNull": true + }, + "attributeType": { + "propertyOrder": 3, + "type": "string", + "enum": [ + "string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + }, + "expression": { + "propertyOrder": 4, + "type": "string", + "title": "Expression" + } + }, + "required": [ + "attributeName", + "attributeType", + "expression" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Python Table Reducer", + "operatorDescription": "Reduce Table to Tuple", + "operatorGroupName": "Python", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Dendrogram", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "threshold": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "description": "Value at which separation of clusters will be made", + "title": "Color Threshold" + }, + "xVal": { + "propertyOrder": 7, + "type": "string", + "description": "The x values of points in dendrogram", + "title": "Value X Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "yVal": { + "propertyOrder": 8, + "type": "string", + "description": "The y value of points in dendrogram", + "title": "Value Y Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Labels": { + "propertyOrder": 9, + "type": "string", + "description": "The label of points in dendrogram", + "title": "Labels", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "xVal", + "yVal", + "Labels" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Dendrogram" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Dendrogram", + "operatorDescription": "Visualize data in a Dendrogram", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "KNNClassifierTrainer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "paraList": { + "propertyOrder": 4, + "type": "array", + "items": { + "$ref": "#/definitions/HyperParameters(SklearnAdvancedKNNParameters)" + }, + "title": "Parameter Setting" + }, + "groundTruthAttribute": { + "propertyOrder": 5, + "type": "string", + "description": "Ground truth attribute column", + "title": "Ground Truth Attribute Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Selected Features": { + "propertyOrder": 8, + "type": "array", + "items": { + "type": "string" + }, + "description": "Features used to train the model", + "title": "Selected Features", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "paraList", + "groundTruthAttribute", + "Selected Features" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "KNNClassifierTrainer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "HyperParameters(SklearnAdvancedKNNParameters)": { + "type": "object", + "additionalProperties": false, + "properties": { + "parameter": { + "propertyOrder": 1, + "type": "string", + "enum": [ + "n_neighbors", + "p", + "weights", + "algorithm", + "leaf_size", + "metric", + "metric_params" + ], + "description": "Choose the name of the parameter", + "title": "Parameter" + }, + "parametersSource": { + "propertyOrder": 2, + "type": "boolean", + "default": false, + "description": "Parameter from workflow", + "title": "Workflow" + }, + "attribute": { + "propertyOrder": 3, + "nullable": true, + "type": "string", + "title": "Attribute", + "autofill": "attributeName", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 1 + }, + "value": { + "propertyOrder": 4, + "nullable": true, + "type": "string", + "title": "Value", + "hideTarget": "parametersSource", + "hideType": "equals", + "hideExpectedValue": "true", + "hideOnNull": true + } + }, + "required": [ + "parameter", + "parametersSource" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "KNN Classifier", + "operatorDescription": "Sklearn KNN Classifier Operator", + "operatorGroupName": "Advanced Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "parameter", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnMultinomialNaiveBayes", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnMultinomialNaiveBayes" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Multinomial Naive Bayes", + "operatorDescription": "Sklearn Multinomial Naive Bayes Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnDecisionTree", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnDecisionTree" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Decision Tree", + "operatorDescription": "Sklearn Decision Tree Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "PostgreSQLSource", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "host": { + "propertyOrder": 4, + "type": "string", + "title": "Host", + "enable-presets": true + }, + "port": { + "propertyOrder": 5, + "type": "string", + "default": "default", + "description": "A port number or 'default'", + "title": "Port", + "enable-presets": true + }, + "database": { + "propertyOrder": 6, + "type": "string", + "title": "Database", + "enable-presets": true + }, + "table": { + "propertyOrder": 7, + "type": "string", + "title": "Table Name", + "enable-presets": true + }, + "username": { + "propertyOrder": 8, + "type": "string", + "title": "Username", + "enable-presets": true + }, + "password": { + "propertyOrder": 9, + "type": "string", + "title": "Password", + "widget": { + "formlyConfig": { + "templateOptions": { + "type": "password" + } + } + } + }, + "limit": { + "propertyOrder": 10, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 11, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + }, + "keywordSearch": { + "propertyOrder": 12, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Keyword Search?", + "toggleHidden": [ + "keywordSearchByColumn", + "keywords" + ] + }, + "keywordSearchByColumn": { + "propertyOrder": 13, + "nullable": true, + "type": "string", + "title": "Keyword Search Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "keywords": { + "propertyOrder": 14, + "nullable": true, + "type": "string", + "description": "E.g. 'sore & throat' for AND; 'sore', 'throat' for OR. See official postgres documents for details.", + "title": "Keywords to Search", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "progressive": { + "propertyOrder": 15, + "nullable": true, + "type": "boolean", + "default": false, + "title": "Progressive?", + "toggleHidden": [ + "batchByColumn", + "min", + "max", + "interval" + ] + }, + "batchByColumn": { + "propertyOrder": 16, + "nullable": true, + "type": "string", + "title": "Batch by Column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "min": { + "propertyOrder": 17, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Min", + "dependOn": "batchByColumn" + }, + "max": { + "propertyOrder": 18, + "nullable": true, + "type": "string", + "default": "auto", + "title": "Max", + "dependOn": "batchByColumn" + }, + "interval": { + "propertyOrder": 19, + "type": "integer", + "default": 1000000000, + "title": "Batch by Interval", + "dependOn": "batchByColumn" + } + }, + "required": [ + "host", + "port", + "database", + "table", + "username", + "password", + "interval" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "PostgreSQLSource" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "PostgreSQL Source", + "operatorDescription": "Read data from a PostgreSQL instance", + "operatorGroupName": "Database Connector", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ArrowSource", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "fileName": { + "propertyOrder": 4, + "type": "string", + "title": "File" + }, + "limit": { + "propertyOrder": 5, + "nullable": true, + "type": "integer", + "description": "max output count", + "title": "Limit" + }, + "offset": { + "propertyOrder": 6, + "nullable": true, + "type": "integer", + "description": "starting point of output", + "title": "Offset" + } + }, + "required": [ + "fileName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ArrowSource" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Arrow File Scan", + "operatorDescription": "Scan data from a Arrow file", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SankeyDiagram", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "Source Attribute": { + "propertyOrder": 6, + "type": "string", + "description": "The source node of the Sankey diagram", + "title": "Source Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Target Attribute": { + "propertyOrder": 7, + "type": "string", + "description": "The target node of the Sankey diagram", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Value Attribute": { + "propertyOrder": 8, + "type": "string", + "description": "The value/volume of the flow between source and target", + "title": "Value Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "Source Attribute", + "Target Attribute", + "Value Attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SankeyDiagram" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Sankey Diagram", + "operatorDescription": "Visualize data using a Sankey diagram", + "operatorGroupName": "Basic", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnSVM", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnSVM" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Support Vector Machine", + "operatorDescription": "Sklearn Support Vector Machine Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnBernoulliNaiveBayes", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnBernoulliNaiveBayes" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Bernoulli Naive Bayes", + "operatorDescription": "Sklearn Bernoulli Naive Bayes Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "ImageVisualizer", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "binaryContent": { + "propertyOrder": 4, + "type": "string", + "description": "The Binary data of the Image", + "title": "image content column", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "binaryContent" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "ImageVisualizer" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Image Visualizer", + "operatorDescription": "visualize image content", + "operatorGroupName": "Media", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnLinearSVM", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnLinearSVM" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Linear Support Vector Machine", + "operatorDescription": "Sklearn Linear Support Vector Machine Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "TextInput", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "textInput": { + "propertyOrder": 4, + "type": "string", + "title": "Text", + "widget": { + "formlyConfig": { + "type": "textarea", + "templateOptions": { + "autosize": true, + "autosizeMinRows": 3 + } + } + } + }, + "attributeType": { + "propertyOrder": 5, + "type": "string", + "default": "string", + "enum": [ + "string", + "single string", + "integer", + "long", + "double", + "boolean", + "timestamp", + "binary" + ], + "title": "Attribute Type" + }, + "attributeName": { + "propertyOrder": 6, + "type": "string", + "default": "line", + "title": "Attribute Name" + }, + "fileScanLimit": { + "propertyOrder": 7, + "nullable": true, + "type": "integer", + "title": "Limit", + "hideTarget": "attributeType", + "hideType": "regex", + "hideExpectedValue": "^binary$|^single string$" + }, + "fileScanOffset": { + "propertyOrder": 8, + "nullable": true, + "type": "integer", + "title": "Offset", + "hideTarget": "attributeType", + "hideType": "regex", + "hideExpectedValue": "^binary$|^single string$" + } + }, + "required": [ + "textInput", + "attributeType", + "attributeName" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "TextInput" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Text Input", + "operatorDescription": "Source data from manually inputted text", + "operatorGroupName": "Data Input", + "inputPorts": [], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "HuggingFaceSpamSMSDetection", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "attribute": { + "propertyOrder": 6, + "type": "string", + "description": "column to perform spam detection on", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "Spam result attribute": { + "propertyOrder": 7, + "type": "string", + "default": "is_spam", + "description": "column name of whether spam or not", + "title": "Spam result attribute" + }, + "Score result attribute": { + "propertyOrder": 8, + "type": "string", + "default": "score", + "description": "column name of Probability for classification", + "title": "Score result attribute" + } + }, + "required": [ + "attribute", + "Spam result attribute", + "Score result attribute" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "HuggingFaceSpamSMSDetection" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Hugging Face Spam Detection", + "operatorDescription": "Spam Detection by SMS Spam Detection Model from Hugging Face", + "operatorGroupName": "Hugging Face", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "QuiverPlot", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "value": { + "enum": [ + "integer", + "long", + "double" + ] + } + }, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "x": { + "propertyOrder": 6, + "type": "string", + "description": "Column for the x-coordinate of the starting point", + "title": "x", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "y": { + "propertyOrder": 7, + "type": "string", + "description": "Column for the y-coordinate of the starting point", + "title": "y", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "u": { + "propertyOrder": 8, + "type": "string", + "description": "Column for the vector component in the x-direction", + "title": "u", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "v": { + "propertyOrder": 9, + "type": "string", + "description": "Column for the vector component in the y-direction", + "title": "v", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "x", + "y", + "u", + "v" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "QuiverPlot" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Quiver Plot", + "operatorDescription": "Visualize vector data in a Quiver Plot", + "operatorGroupName": "Scientific", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 2, + "index": 2, + "name": "SINGLE_SNAPSHOT", + "singleSnapshot": true, + "setDelta": false, + "unrecognized": false, + "setSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "SklearnNearestCentroid", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "target": { + "propertyOrder": 4, + "type": "string", + "description": "Attribute in your dataset corresponding to target.", + "title": "Target Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "countVectorizer": { + "propertyOrder": 5, + "type": "boolean", + "default": false, + "description": "Convert a collection of text documents to a matrix of token counts.", + "title": "Count Vectorizer" + }, + "text": { + "propertyOrder": 6, + "nullable": true, + "type": "string", + "description": "Attribute in your dataset with text to vectorize.", + "title": "Text Attribute", + "autofill": "attributeName", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false", + "autofillAttributeOnPort": 0 + }, + "tfidfTransformer": { + "propertyOrder": 7, + "type": "boolean", + "default": false, + "description": "Transform a count matrix to a normalized tf or tf-idf representation.", + "title": "Tfidf Transformer", + "hideTarget": "countVectorizer", + "hideType": "equals", + "hideExpectedValue": "false" + } + }, + "required": [ + "target", + "countVectorizer", + "tfidfTransformer" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "SklearnNearestCentroid" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Nearest Centroid", + "operatorDescription": "Sklearn Nearest Centroid Operator", + "operatorGroupName": "Sklearn", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "training", + "allowMultiLinks": false, + "dependencies": [] + }, + { + "id": { + "id": 1, + "internal": false + }, + "displayName": "testing", + "allowMultiLinks": false, + "dependencies": [ + { + "id": 0, + "internal": false + } + ] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": true, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + }, + { + "operatorType": "Aggregate", + "jsonSchema": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "additionalProperties": false, + "properties": { + "dummyPropertyList": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/DummyProperties" + }, + "description": "Add dummy property if needed", + "title": "Dummy Property List" + }, + "aggregations": { + "propertyOrder": 6, + "type": "array", + "minItems": 1, + "maxItems": 2147483647, + "items": { + "$ref": "#/definitions/AggregationOperation" + }, + "description": "multiple aggregation functions", + "title": "Aggregations" + }, + "groupByKeys": { + "propertyOrder": 7, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "description": "group by columns", + "title": "Group By Keys", + "autofill": "attributeNameList", + "autofillAttributeOnPort": 0 + } + }, + "required": [ + "aggregations" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "operatorType", + "value": "Aggregate" + } + }, + "definitions": { + "DummyProperties": { + "type": "object", + "additionalProperties": false, + "properties": { + "dummyProperty": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Dummy Property" + }, + "dummyValue": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Dummy Value" + } + } + }, + "PortDescription": { + "type": "object", + "additionalProperties": false, + "properties": { + "portID": { + "propertyOrder": 1, + "nullable": true, + "type": "string", + "title": "Port ID" + }, + "displayName": { + "propertyOrder": 2, + "nullable": true, + "type": "string", + "title": "Display Name" + }, + "allowMultiInputs": { + "propertyOrder": 3, + "type": "boolean", + "title": "Allow Multi Inputs" + }, + "isDynamicPort": { + "propertyOrder": 4, + "type": "boolean", + "title": "Is Dynamic Port" + }, + "partitionRequirement": { + "propertyOrder": 5, + "nullable": true, + "oneOf": [ + { + "$ref": "#/definitions/HashPartition" + }, + { + "$ref": "#/definitions/RangePartition" + }, + { + "$ref": "#/definitions/SinglePartition" + }, + { + "$ref": "#/definitions/BroadcastPartition" + }, + { + "$ref": "#/definitions/UnknownPartition" + } + ], + "title": "Partition Requirement" + }, + "dependencies": { + "propertyOrder": 6, + "nullable": true, + "type": "array", + "items": { + "$ref": "#/definitions/Object" + }, + "title": "Dependencies" + } + }, + "required": [ + "allowMultiInputs", + "isDynamicPort" + ] + }, + "HashPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "hash" + ], + "default": "hash", + "options": { + "hidden": true + } + }, + "hashAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Hash Attribute Names" + } + }, + "title": "hash", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "hash" + } + } + }, + "RangePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "range" + ], + "default": "range", + "options": { + "hidden": true + } + }, + "rangeAttributeNames": { + "propertyOrder": 1, + "nullable": true, + "type": "array", + "items": { + "type": "string" + }, + "title": "Range Attribute Names" + }, + "rangeMin": { + "propertyOrder": 2, + "type": "integer", + "title": "Range Min" + }, + "rangeMax": { + "propertyOrder": 3, + "type": "integer", + "title": "Range Max" + } + }, + "title": "range", + "required": [ + "type", + "rangeMin", + "rangeMax" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "range" + } + } + }, + "SinglePartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "single" + ], + "default": "single", + "options": { + "hidden": true + } + } + }, + "title": "single", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "single" + } + } + }, + "BroadcastPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "broadcast" + ], + "default": "broadcast", + "options": { + "hidden": true + } + } + }, + "title": "broadcast", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "broadcast" + } + } + }, + "UnknownPartition": { + "type": "object", + "additionalProperties": false, + "properties": { + "type": { + "type": "string", + "enum": [ + "none" + ], + "default": "none", + "options": { + "hidden": true + } + } + }, + "title": "none", + "required": [ + "type" + ], + "options": { + "multiple_editor_select_via_property": { + "property": "type", + "value": "none" + } + } + }, + "Object": { + "type": "object", + "additionalProperties": false, + "properties": {} + }, + "AggregationOperation": { + "type": "object", + "additionalProperties": false, + "attributeTypeRules": { + "attribute": { + "allOf": [ + { + "if": { + "aggFunction": { + "valEnum": [ + "sum", + "average", + "min", + "max" + ] + } + }, + "then": { + "enum": [ + "integer", + "long", + "double", + "timestamp" + ] + } + }, + { + "if": { + "aggFunction": { + "valEnum": [ + "concat" + ] + } + }, + "then": { + "enum": [ + "string" + ] + } + } + ] + } + }, + "properties": { + "aggFunction": { + "propertyOrder": 1, + "type": "string", + "enum": [ + "sum", + "count", + "average", + "min", + "max", + "concat" + ], + "description": "sum, count, average, min, max, or concat", + "title": "Aggregate Func" + }, + "attribute": { + "propertyOrder": 2, + "type": "string", + "description": "column to calculate average value", + "title": "Attribute", + "autofill": "attributeName", + "autofillAttributeOnPort": 0 + }, + "result attribute": { + "propertyOrder": 3, + "type": "string", + "minLength": 1, + "description": "column name of average result", + "title": "Result attribute" + } + }, + "required": [ + "aggFunction", + "attribute", + "result attribute" + ] + } + } + }, + "additionalMetadata": { + "userFriendlyName": "Aggregate", + "operatorDescription": "Calculate different types of aggregation values", + "operatorGroupName": "Aggregate", + "inputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "allowMultiLinks": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "id": { + "id": 0, + "internal": false + }, + "displayName": "", + "blocking": false, + "mode": { + "value": 0, + "index": 0, + "name": "SET_SNAPSHOT", + "setSnapshot": true, + "setDelta": false, + "unrecognized": false, + "singleSnapshot": false + } + } + ], + "dynamicInputPorts": false, + "dynamicOutputPorts": false, + "supportReconfiguration": false, + "allowPortCustomization": false + }, + "operatorVersion": "N/A" + } + ], + "groups": [ + { + "groupName": "Data Input", + "children": null + }, + { + "groupName": "Database Connector", + "children": null + }, + { + "groupName": "Search", + "children": null + }, + { + "groupName": "Data Cleaning", + "children": [ + { + "groupName": "Join", + "children": null + }, + { + "groupName": "Aggregate", + "children": null + }, + { + "groupName": "Sort", + "children": null + } + ] + }, + { + "groupName": "Machine Learning", + "children": [ + { + "groupName": "Sklearn", + "children": null + }, + { + "groupName": "Advanced Sklearn", + "children": null + }, + { + "groupName": "Hugging Face", + "children": null + }, + { + "groupName": "Machine Learning General", + "children": null + } + ] + }, + { + "groupName": "Utilities", + "children": null + }, + { + "groupName": "External API", + "children": null + }, + { + "groupName": "User-defined Functions", + "children": [ + { + "groupName": "Python", + "children": null + }, + { + "groupName": "Java", + "children": null + }, + { + "groupName": "R", + "children": null + } + ] + }, + { + "groupName": "Visualization", + "children": [ + { + "groupName": "Basic", + "children": null + }, + { + "groupName": "Statistical", + "children": null + }, + { + "groupName": "Scientific", + "children": null + }, + { + "groupName": "Financial", + "children": null + }, + { + "groupName": "Media", + "children": null + }, + { + "groupName": "Advanced", + "children": null + } + ] + }, + { + "groupName": "Control Block", + "children": null + } + ] +} \ No newline at end of file diff --git a/core/suggestion-service/files/operator_type_and_descriptions.md b/core/suggestion-service/files/operator_type_and_descriptions.md new file mode 100644 index 00000000000..49215004f8d --- /dev/null +++ b/core/suggestion-service/files/operator_type_and_descriptions.md @@ -0,0 +1,115 @@ +# Operator Types and Descriptions + +- IntervalJoin: Join two inputs with left table join key in the range of [right table join key, right table join key + constant value] +- DotPlot: Visualize data using a dot plot +- CartesianProduct: Append fields together to get the cartesian product of two inputs +- HuggingFaceSentimentAnalysis: Analyzing Sentiments with a Twitter-Based Model from Hugging Face +- PythonUDFSourceV2: User-defined function operator in Python script +- TwitterFullArchiveSearch: Retrieve data from Twitter Full Archive Search API +- SklearnLogisticRegressionCV: Sklearn Logistic Regression Cross Validation Operator +- JSONLFileScan: Scan data from a JSONL file +- CandlestickChart: Visualize data in a Candlestick Chart +- ReservoirSampling: Reservoir Sampling with k items being kept randomly +- ScatterMatrixChart: Visualize datasets in a Scatter Matrix +- SklearnKNN: Sklearn K-nearest Neighbors Operator +- SklearnProbabilityCalibration: Sklearn Probability Calibration Operator +- SortPartitions: Sort Partitions +- DumbbellPlot: Visualize data in a Dumbbell Plots. A dumbbell plots (also known as a lollipop chart) is typically used to compare two distinct values or time points for the same entity. +- If: If +- SklearnSDG: Sklearn Stochastic Gradient Descent Operator +- URLVisualizer: Render the content of URL +- Dummy: A dummy operator used as a placeholder. +- HuggingFaceTextSummarization: Summarize the given text content with a mini2bert pre-trained model from Hugging Face +- Union: Unions the output rows from multiple input operators +- SklearnGradientBoosting: Sklearn Gradient Boosting Operator +- KNNRegressorTrainer: Sklearn KNN Regressor Operator +- RUDFSource: User-defined function operator in R script +- HuggingFaceIrisLogisticRegression: Predict whether an iris is an Iris-setosa using a pre-trained logistic regression model +- ContinuousErrorBands: Visualize error or uncertainty along a continuous line +- TwitterSearch: Retrieve data from Twitter Search API +- SklearnPassiveAggressive: Sklearn Passive Aggressive Operator +- HTMLVisualizer: Render the result of HTML content +- SklearnComplementNaiveBayes: Sklearn Complement Naive Bayes Operator +- URLFetcher: Fetch the content of a single url +- JavaUDF: User-defined function operator in Java script +- PieChart: Visualize data in a Pie Chart +- DictionaryMatcher: Matches tuples if they appear in a given dictionary +- UnnestString: Unnest the string values in the column separated by a delimiter to multiple values +- BubbleChart: a 3D Scatter Plot; Bubbles are graphed using x and y labels, and their sizes determined by a z-value. +- RedditSearch: Search for recent posts with python-wrapped Reddit API, PRAW +- SVRTrainer: Sklearn SVM Regressor Operator +- RUDF: User-defined function operator in R script +- BoxViolinPlot: Visualize data using either a Box Plot or a Violin Plot. Box plots are drawn as a box with a vertical line down the middle which is mean value, and has horizontal lines attached to each side (known as “whiskers”). Violin plots provide more detail by showing a smoothed density curve on each side, and also include a box plot inside for comparison. +- SklearnAdaptiveBoosting: Sklearn Adaptive Boosting Operator +- Scatterplot: View the result in a scatterplot +- SklearnPerceptron: Sklearn Linear Perceptron Operator +- KeywordSearch: Search for keyword(s) in a string column +- PythonUDFV2: User-defined function operator in Python script +- SklearnLogisticRegression: Sklearn Logistic Regression Operator +- SklearnRandomForest: Sklearn Random Forest Operator +- TypeCasting: Cast between types +- SklearnGaussianNaiveBayes: Sklearn Gaussian Naive Bayes Operator +- AsterixDBSource: Read data from a AsterixDB instance +- DualInputPortsPythonUDFV2: User-defined function operator in Python script +- Histogram: Visualize data in a Histogram Chart +- SklearnDummyClassifier: Sklearn Dummy Classifier Operator +- Distinct: Remove duplicate tuples +- NetworkGraph: Visualize data in a network graph +- WaterfallChart: Visualize data as a waterfall chart +- Limit: Limit the number of output rows +- Scorer: Scorer for machine learning models +- SklearnExtraTrees: Sklearn Extra Trees Operator +- FileScan: Scan data from a file +- GanttChart: A Gantt chart is a type of bar chart that illustrates a project schedule. The chart lists the tasks to be performed on the vertical axis, and time intervals on the horizontal axis. The width of the horizontal bars in the graph shows the duration of each activity. +- TernaryPlot: Points are graphed on a Ternary Plot using 3 specified data fields +- SVCTrainer: Sklearn SVM Classifier Operator +- SklearnLinearRegression: Sklearn Linear Regression Operator +- MySQLSource: Read data from a MySQL instance +- CSVOldFileScan: Scan data from a CSVOld file +- CSVFileScan: Scan data from a CSV file +- FunnelPlot: Visualize data in a Funnel Plot +- Projection: Keeps or drops the column +- Filter: Performs a filter operation +- SklearnRidge: Sklearn Ridge Regression Operator +- Intersect: Take the intersect of two inputs +- SklearnPrediction: Skleanr Prediction Operator +- SymmetricDifference: find the symmetric difference (the set of elements which are in either of the sets, but not in their intersection) of two inputs +- FigureFactoryTable: Visualize data in a figure factory table +- FilledAreaPlot: Visualize data in filled area plot +- SklearnRidgeCV: Sklearn Ridge Regression Cross Validation Operator +- IcicleChart: Visualize hierarchical data from root to leaves +- Regex: Search a regular expression in a string column +- HeatMap: Visualize data in a HeatMap Chart +- TablesPlot: Visualize data in a table chart. +- HierarchyChart: Visualize data in hierarchy +- SklearnExtraTree: Sklearn Extra Tree Operator +- Sort: Sort based on the columns and sorting methods +- Scatter3DChart: Visualize data in a Scatter3D Plot +- SklearnBagging: Sklearn Bagging Operator +- Difference: find the set difference of two inputs +- ContourPlot: Displays terrain or gradient variations in a Contour Plot +- PythonLambdaFunction: Modify or add a new column with more ease +- WordCloud: Generate word cloud for texts +- LineChart: View the result in line chart +- RandomKSampling: random sampling with given percentage +- Split: Split data to two different ports +- SklearnMultiLayerPerceptron: Sklearn Multi-layer Perceptron Operator +- BarChart: Visualize data in a Bar Chart +- HashJoin: join two inputs +- PythonTableReducer: Reduce Table to Tuple +- Dendrogram: Visualize data in a Dendrogram +- KNNClassifierTrainer: Sklearn KNN Classifier Operator +- SklearnMultinomialNaiveBayes: Sklearn Multinomial Naive Bayes Operator +- SklearnDecisionTree: Sklearn Decision Tree Operator +- PostgreSQLSource: Read data from a PostgreSQL instance +- ArrowSource: Scan data from a Arrow file +- SankeyDiagram: Visualize data using a Sankey diagram +- SklearnSVM: Sklearn Support Vector Machine Operator +- SklearnBernoulliNaiveBayes: Sklearn Bernoulli Naive Bayes Operator +- ImageVisualizer: visualize image content +- SklearnLinearSVM: Sklearn Linear Support Vector Machine Operator +- TextInput: Source data from manually inputted text +- HuggingFaceSpamSMSDetection: Spam Detection by SMS Spam Detection Model from Hugging Face +- QuiverPlot: Visualize vector data in a Quiver Plot +- SklearnNearestCentroid: Sklearn Nearest Centroid Operator +- Aggregate: Calculate different types of aggregation values diff --git a/core/suggestion-service/files/output_format.json b/core/suggestion-service/files/output_format.json new file mode 100644 index 00000000000..e98f4295f91 --- /dev/null +++ b/core/suggestion-service/files/output_format.json @@ -0,0 +1,86 @@ +{ + "type": "array", + "items": { + "type": "object", + "properties": { + "suggestion": { + "type": "string", + "description": "A clear description of the improvement" + }, + "suggestionType": { + "type": "string", + "description": "the type of the proposed suggestion. 2 POSSIBLE VALUES: fix, improve" + }, + "changes": { + "type": "object", + "properties": { + "operatorsToAdd": { + "type": "array", + "description": "Operators to add or update. If the operatorID exists in the original workflow, treat this as an update.", + "items": { + "type": "object", + "properties": { + "operatorType": { + "type": "string", + "description": "The type of operator to be added or updated" + }, + "operatorID": { + "type": "string", + "description": "A unique identifier, may match existing operator in prompt" + }, + "operatorProperties": { + "type": "object", + "description": "Properties to assign or update for the operator", + "additionalProperties": true + }, + "customDisplayName": { + "type": "string", + "description": "Optional logical name describing function" + } + }, + "required": ["operatorType", "operatorID", "operatorProperties"] + } + }, + "linksToAdd": { + "type": "array", + "items": { + "type": "object", + "properties": { + "linkID": { + "type": "string", + "description": "A unique ID for the link" + }, + "source": { + "type": "object", + "properties": { + "operatorID": { "type": "string" }, + "portID": { "type": "string", "pattern": "^output-\\d+$" } + }, + "required": ["operatorID", "portID"] + }, + "target": { + "type": "object", + "properties": { + "operatorID": { "type": "string" }, + "portID": { "type": "string", "pattern": "^input-\\d+$" } + }, + "required": ["operatorID", "portID"] + } + }, + "required": ["linkID", "source", "target"] + } + }, + "operatorsToDelete": { + "type": "array", + "items": { + "type": "string", + "description": "List of operator IDs to delete" + } + } + }, + "required": ["operatorsToAdd", "linksToAdd"] + } + }, + "required": ["suggestion", "changes"] + } +} diff --git a/core/suggestion-service/files/texera_paper.pdf b/core/suggestion-service/files/texera_paper.pdf new file mode 100644 index 00000000000..49a64b0d00b Binary files /dev/null and b/core/suggestion-service/files/texera_paper.pdf differ diff --git a/core/suggestion-service/files/texera_python_udf.md b/core/suggestion-service/files/texera_python_udf.md new file mode 100644 index 00000000000..8252a94af36 --- /dev/null +++ b/core/suggestion-service/files/texera_python_udf.md @@ -0,0 +1,159 @@ +## What is Python UDF +User-defined Functions (UDFs) provide a means to incorporate custom logic into Texera. Texera offers comprehensive Python UDF APIs, enabling users to accomplish various tasks. This guide will delve into the usage of UDFs, breaking down the process step by step. + + +*** + + +## UDF UI and Editor + + +The UDF operator offers the following interface, requiring the user to provide the following inputs: `Python code`, `worker count`, and `output schema`. + +

    +Screenshot 2023-07-04 at 12 51 37 +

    + +- Screenshot 2023-07-04 at 13 25 59 Users can click on the "Edit code content" button to open the UDF code editor, where they can enter their custom Python code to define the desired operator. + +- Screenshot 2023-07-04 at 13 27 22 Users have the flexibility to adjust the parallelism of the UDF operator by modifying the number of workers. The engine will then create the corresponding number of workers to execute the same operator in parallel. + +- Screenshot 2023-07-04 at 13 27 29 Users need to provide the output schema of the UDF operator, which describes the output data's fields. + - The option `Retain input columns` allows users to include the input schema as the foundation for the output schema. + - The `Extra output column(s)` list allows users to define additional fields that should be included in the output schema. + +
    +
    +
    + +- Screenshot 2023-07-04 at 13 04 31 _Optionally_, users can click on the pencil icon located next to the operator name to make modifications to the name of the operator. + + +*** + +## Operator Definition + +### Iterator-based operator +In Texera, all operators are implemented as iterators, including Python UDFs. +Concepturally, a defined operator is executed as: + +```python +operator = UDF() # initialize a UDF operator + +... # some other initialization logic + +# the main process loop +while input_stream.has_more(): + input_data = next_data() + output_iterator = operator.process(input_data) + for output_data in output_iterator: + send(output_data) + +... # some cleanup logic + +``` + +### Operator Life Cycle +The complete life cycle of a UDF operator consists of the following APIs: +1. `open() -> None` Open a context of the operator. Usually it can be used for loading/initiating some resources, such as a file, a model, or an API client. It will be invoked once per operator. +2. `process(data, port: int) -> Iterator[Optional[data]]` Process an input data from the given port, returning an iterator of optional data as output. It will be invoked once for every unit of data. +3. `on_finish(port: int) -> Iterator[Optional[data]]` Callback when one input port is exhausted, returning an iterator of optional data as output. It will be invoked once per port. +4. `close() -> None` Close the context of the operator. It will be invoked once per operator. + + +### Process Data APIs +There are three APIs to process the data in different units. + +1. Tuple API. + +```python + +class ProcessTupleOperator(UDFOperatorV2): + + def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]: + yield tuple_ + +``` +Tuple API takes one input tuple from a port at a time. It returns an iterator of optional `TupleLike` instances. A `TupleLike` is any data structure that supports key-value pairs, such as `pytexera.Tuple`, `dict`, `defaultdict`, `NamedTuple`, etc. + +Tuple API is useful for implementing functional operations which are applied to tuples one by one, such as map, reduce, and filter. + +2. Table API. +```python + +class ProcessTableOperator(UDFTableOperator): + + def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]: + yield table +``` +Table API consumes a `Table` at a time, which consists of all the tuples from a port. It returns an iterator of optional `TableLike` instances. A `TableLike ` is a collection of `TupleLike`, and currently, we support `pytexera.Table` and `pandas.DataFrame` as a `TableLike` instance. More flexible types will be supported down the road. + +Table API is useful for implementing blocking operations that will consume all the data from one port, such as join, sort, and machine learning training. + +3. Batch API. +```python + +class ProcessBatchOperator(UDFBatchOperator): + + BATCH_SIZE = 10 + + def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]: + yield batch +``` +Batch API consumes a batch of tuples at a time. Similar to `Table`, a `Batch` is also a collection of `Tuple`s; however, its size is defined by the `BATCH_SIZE`, and one port can have multiple batches. It returns an iterator of optional `BatchLike` instances. A `BatchLike ` is a collection of `TupleLike`, and currently, we support `pytexera.Batch` and `pandas.DataFrame` as a `BatchLike` instance. More flexible types will be supported down the road. + +The Batch API serves as a hybrid API combining the features of both the Tuple and Table APIs. It is particularly valuable for striking a balance between time and space considerations, offering a trade-off that optimizes efficiency. + +_All three APIs can return an empty iterator by `yield None`._ + +### Schemas + +A UDF has an input Schema and an output Schema. The input schema is determined by the upstream operator's output schema and the engine will make sure the input data (tuple, table, or batch) matches the input schema. On the other hand, users are required to define the output schema of the UDF, and it is the user's responsibility to make sure the data output from the UDF matches the defined output schema. + +### Ports + +- Input ports: +A UDF can take zero, one or multiple input ports, different ports can have different input schemas. Each port can take in multiple links, as long as they share the same schema. + +- Output ports: +Currently, a UDF can only have exactly one output port. This means it cannot be used as a terminal operator (i.e., operator without output ports), or have more than one output port. + +#### 1-out UDF + +This UDF has zero input port and one output port. It is considered as a source operator (operator that produces data without an upstream). It has a special API: +```python + +class GenerateOperator(UDFSourceOperator): + + @overrides + def produce(self) -> Iterator[Union[TupleLike, TableLike, None]]: + yield +``` + +This `produce()` API returns an iterator of `TupleLike`, `TableLike`, or simply `None`. + +See [Generator Operator](https://github.com/Texera/texera/blob/master/core/amber/src/main/python/pytexera/udf/examples/generator_operator.py) for an example of 1-out UDF. + + +#### 2-in UDF + +This UDF has two input ports, namely `model` port and `tuples` port. The `tuples` port depends on the `model` port, which means that during the execution, the `model` port will execute first, and the `tuples` port will start after the `model` port consumes all its input data. +This dependency is particularly useful to implement machine learning inference operators, where a machine learning model is sent into the 2-in UDF through the `model` port, and becomes an operator state, then the tuples are coming in through the `tuples` port to be processed by the model. + +An example of 2-in UDF: +``` +class SVMClassifier(UDFOperatorV2): + + + @overrides + def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]: + + if port == 0: # models port + self.model = tuple_['model'] + + else: # tuples port + tuple_['pred'] = self.model.predict(tuple_['text']) + yield tuple_ +``` + +_Currently, in 2-in UDF, "Retain input columns" will retain only the `tuples` port's input schema._ \ No newline at end of file diff --git a/core/suggestion-service/llm_agent/__init__.py b/core/suggestion-service/llm_agent/__init__.py new file mode 100644 index 00000000000..688f80a8134 --- /dev/null +++ b/core/suggestion-service/llm_agent/__init__.py @@ -0,0 +1,7 @@ +"""LLM Agent module for calling language models.""" + +from llm_agent.base import LLMAgent, LLMAgentFactory +from llm_agent.openai_agent import OpenAIAgent +from llm_agent.anthropic_agent import AnthropicAgent + +__all__ = ["LLMAgent", "LLMAgentFactory", "OpenAIAgent", "AnthropicAgent"] diff --git a/core/suggestion-service/llm_agent/anthropic_agent.py b/core/suggestion-service/llm_agent/anthropic_agent.py new file mode 100644 index 00000000000..e346298de1d --- /dev/null +++ b/core/suggestion-service/llm_agent/anthropic_agent.py @@ -0,0 +1,135 @@ +"""Anthropic Claude LLM agent.""" + +import os +import json +from typing import Dict, Any, List, Optional + +import anthropic +from anthropic import Anthropic + +from llm_agent.base import LLMAgent, LLMAgentFactory +from model.llm.suggestion import SuggestionList + + +@LLMAgentFactory.register("anthropic") +class AnthropicAgent(LLMAgent): + """ + Implementation of the LLM agent interface using Anthropic's Claude API. + + This agent supports different Claude models, such as: + - claude-3-opus-20240229 + - claude-3-sonnet-20240229 + - claude-3-haiku-20240307 + - claude-2.1 + - claude-2.0 + """ + + def __init__( + self, model: str = "claude-3-opus-20240229", api_key: Optional[str] = None + ): + """ + Initialize the Anthropic Claude agent. + + Args: + model: The Claude model to use + api_key: The Anthropic API key (if None, uses ANTHROPIC_API_KEY environment variable) + """ + self.model = model + self.client = Anthropic(api_key=api_key or os.environ.get("ANTHROPIC_API_KEY")) + + def generate_suggestions( + self, + prompt: str, + max_suggestions: int = 3, + temperature: float = 0.7, + max_tokens: Optional[int] = None, + **kwargs, + ) -> SuggestionList: + """ + Generate workflow suggestions using Anthropic's Claude API. + + Args: + prompt: The natural language prompt describing the workflow + max_suggestions: Maximum number of suggestions to generate + temperature: Sampling temperature (0.0-1.0) where lower is more deterministic + max_tokens: Maximum number of tokens to generate (if None, defaults to 4096) + **kwargs: Additional Claude-specific parameters + + Returns: + A list of suggestion dictionaries formatted according to the interface + """ + # Enhance the prompt with instruction about the output format + system_prompt = self._create_system_prompt(max_suggestions) + + try: + response = self.client.messages.create( + model=self.model, + system=system_prompt, + messages=[{"role": "user", "content": prompt}], + temperature=temperature, + max_tokens=max_tokens or 4096, + **kwargs, + ) + + # Extract content from the response + raw_content = response.content[0].text + + return raw_content[:max_suggestions] + + except Exception as e: + print(f"Error generating suggestions with Anthropic Claude: {str(e)}") + return [] + + def _create_system_prompt(self, max_suggestions: int) -> str: + """Create the system prompt for the Claude model.""" + return f"""You are an AI assistant that helps users improve their Texera workflows by suggesting useful modifications. + +Analyze the provided workflow description and generate {max_suggestions} suggestions to improve it. +Your suggestions should address common issues, optimizations, or additional useful features. + +For each suggestion, provide: +1. A brief natural language description of the recommendation +2. Detailed changes required to implement the suggestion + +Structure each suggestion as a valid JSON object with the following format: +{{ + "suggestion": "A clear description of the improvement", + "changes": {{ + "operatorsToAdd": [ + {{ + "operatorType": "TypeOfOperator", + "operatorID": "OperatorType-operator-UUID", + "operatorProperties": {{ + "property1": "value1", + "property2": "value2" + }}, + "customDisplayName": "Logical name describing function" + }} + ], + "linksToAdd": [ + {{ + "linkID": "link-UUID", + "source": {{ + "operatorID": "SourceOperatorID", + "portID": "output-N" + }}, + "target": {{ + "operatorID": "TargetOperatorID", + "portID": "input-N" + }} + }} + ], + "operatorsToDelete": ["OperatorID1", "OperatorID2"], + "operatorPropertiesToChange": [ + {{ + "operatorID": "ExistingOperatorID", + "properties": {{ + "propertyToChange": "newValue" + }} + }} + ] + }} +}} + +Output all suggestions as a valid JSON array and ensure each suggestion follows the exact format above. +Be sure to only generate valid JSON in your response.""" diff --git a/core/suggestion-service/llm_agent/base.py b/core/suggestion-service/llm_agent/base.py new file mode 100644 index 00000000000..d88acc750c9 --- /dev/null +++ b/core/suggestion-service/llm_agent/base.py @@ -0,0 +1,138 @@ +"""Base LLM agent interface""" + +from abc import ABC, abstractmethod +from typing import Dict, Any, List, Optional, Type +import os + +from model.llm import SuggestionPrompt +from model.llm.prompt import DataCleaningSuggestionPrompt +from model.llm.suggestion import SuggestionList, DataCleaningSuggestionList + + +class LLMAgent(ABC): + """ + Abstract base class for LLM agents that generate workflow suggestions. + All LLM providers (OpenAI, Anthropic, etc.) should implement this interface. + """ + + def __init__(self, model: str = None, api_key: str = None): + """ + Initialize the LLM agent. + + Args: + model: The model to use for generation + api_key: The API key for the LLM service + """ + self.model = model + self.api_key = api_key + + @abstractmethod + def generate_suggestions( + self, + prompt: SuggestionPrompt, + temperature: float = 0.7, + max_tokens: Optional[int] = None, + **kwargs, + ) -> SuggestionList: + """ + Generate workflow suggestions using the LLM provider. + + Args: + prompt: The natural language prompt describing the workflow + max_suggestions: Maximum number of suggestions to generate + temperature: Sampling temperature (0.0-1.0) where lower is more deterministic + max_tokens: Maximum number of tokens to generate + **kwargs: Additional provider-specific parameters + + Returns: + A list of suggestion dictionaries with the following format: + { + "suggestion": str, # Natural language description of the suggestion + "changes": { + "operatorsToAdd": List[Dict], # Operators to add to the workflow + "linksToAdd": List[Dict], # Links to add between operators + "operatorsToDelete": List[str], # Operator IDs to delete + "operatorPropertiesToChange": List[Dict], # Properties to change on existing operators + } + } + """ + pass + + @abstractmethod + def generate_data_cleaning_suggestions( + self, + prompt: DataCleaningSuggestionPrompt, + temperature: float = 0.7, + max_tokens: Optional[int] = None, + **kwargs, + ) -> DataCleaningSuggestionList: + """ + generate the natural language plan of cleaning the given column + """ + pass + + +class LLMAgentFactory: + """Factory for creating LLM agents based on provider name.""" + + _agent_registry: Dict[str, Type[LLMAgent]] = {} + + @classmethod + def register(cls, name: str): + """ + Decorator to register an LLM agent class with the factory. + + Args: + name: The name to register the agent under + + Returns: + A decorator function + """ + + def decorator(agent_class): + cls._agent_registry[name.lower()] = agent_class + return agent_class + + return decorator + + @classmethod + def create(cls, provider: str, model: str, **kwargs) -> LLMAgent: + """ + Create an LLM agent instance based on the provider. + + Args: + provider: The LLM provider to use (e.g., 'openai', 'anthropic') + model: The model to use (if None, will use default for the provider) + api_key: The API key (if None, will use environment variable) + **kwargs: Additional parameters to pass to the agent constructor + + Returns: + An instance of the appropriate LLM agent + + Raises: + ValueError: If the provider is not registered or if required configuration is missing + """ + provider = provider.lower() + + if provider not in cls._agent_registry: + raise ValueError( + f"LLM provider '{provider}' is not supported. Available providers: {list(cls._agent_registry.keys())}" + ) + + # Get the agent class from the registry + agent_class = cls._agent_registry[provider] + + # Determine API key from environment if not provided + if provider == "openai": + api_key = os.environ.get("OPENAI_API_KEY") + elif provider == "anthropic": + api_key = os.environ.get("ANTHROPIC_API_KEY") + + # Check if API key is available + if not api_key: + raise ValueError( + f"API key for {provider} not provided and not found in environment variables" + ) + + # Create and return the agent instance + return agent_class(model=model, api_key=api_key, **kwargs) diff --git a/core/suggestion-service/llm_agent/openai_agent.py b/core/suggestion-service/llm_agent/openai_agent.py new file mode 100644 index 00000000000..c960f103bcc --- /dev/null +++ b/core/suggestion-service/llm_agent/openai_agent.py @@ -0,0 +1,432 @@ +import json +import uuid +from typing import Dict, Any, List, Optional + +from openai import OpenAI +from llm_agent.base import LLMAgent, LLMAgentFactory +from llm_agent.utils.operator_metadata_converter import extract_json_schemas +from model.llm import SuggestionPrompt +from model.llm.interpretation import BaseInterpretation +from model.llm.prompt import DataCleaningSuggestionPrompt +from model.llm.suggestion import SuggestionList, Suggestion, DataCleaningSuggestionList + + +@LLMAgentFactory.register("openai") +class OpenAIAgent(LLMAgent): + """ + Implementation of the LLM agent using OpenAI's `responses.create` API + to both generate and sanitize workflow suggestions. + """ + + def __init__( + self, + model, + tools, + api_key, + project, + organization, + use_function_calls: bool = True, + ): + """ + Initialize the OpenAI agent. + + Args: + model: OpenAI model name (e.g., "gpt-4o-2024-08-06") + api_key: API key for OpenAI + project: Project ID for usage (optional) + """ + super().__init__(model, api_key) + self.model = model + self.tools = tools + self.client = OpenAI( + api_key=api_key, project=project, organization=organization + ) + + # Load first phase instruction + with open("files/instruction_for_suggestion.md", "r") as f: + self.instruction_for_suggestion = f.read() + + # Load sanitizer instruction + with open("files/instruction_for_sanitizor.md", "r") as f: + self.instruction_for_sanitizor = f.read() + + with open("files/instruction_for_function_call.md", "r") as f: + self.instruction_for_suggestion_fc = f.read() + + with open("files/instruction_for_data_cleaning_agent.md", "r") as f: + self.instruction_for_dc_suggestion = f.read() + + self.function_tools = [ + { + "type": "function", + "name": "extract_json_schemas", + "description": ( + "Return a list of JSON schemas for given Texera operator types. " + ), + "parameters": { + "type": "object", + "properties": { + "operator_types": { + "type": "array", + "description": "List of operatorType names to fetch.", + "items": {"type": "string"}, + }, + }, + "required": ["operator_types"], + "additionalProperties": False, + }, + "strict": True, + } + ] + + self.use_function_calls = use_function_calls + self.enable_llm_sanitizor = False + + def generate_suggestions( + self, + prompt: SuggestionPrompt, + temperature: float = 0.7, + max_tokens: Optional[int] = None, + **kwargs, + ) -> SuggestionList: + if not self.use_function_calls: + # fall back to the old single‑shot method + return self._generate_suggestions_one_shot(prompt, temperature, max_tokens) + else: + # NEW two‑turn method + return self._generate_suggestions_with_function_calls( + prompt, temperature, max_tokens + ) + + def _generate_suggestions_one_shot(self, prompt, temperature, max_tokens): + """ + Generate workflow suggestions using OpenAI's `responses.create` endpoint with schema enforcement. + + Args: + prompt: Workflow description + temperature: Sampling temperature + max_tokens: Maximum tokens allowed in output + + Returns: + A list of sanitized suggestion dicts. + """ + try: + prompt_json: str = prompt.model_dump_json(indent=2) + # Step 1: Generate raw suggestions + raw_response = self.client.responses.create( + model=self.model, + instructions=self.instruction_for_suggestion, + input=prompt_json, + tools=self.tools, + # text_format=SuggestionList + text={ + "format": { + "type": "json_schema", + "name": "workflow_suggestions", + "schema": SuggestionList.model_json_schema(), + "strict": False, # Allow flexible output in generation + } + }, + ) + + raw_suggestions = json.loads(raw_response.output_text)["suggestions"] + with open("raw_suggestions_debug.json", "w") as f: + json.dump(raw_suggestions, f, indent=2) + # Step 2: Sanitize the suggestions + sanitized_suggestions = self._sanitize_suggestions( + raw_suggestions, prompt.workflowInterpretation + ) + + return sanitized_suggestions + + except Exception as e: + print(f"Error generating suggestions: {e}") + return SuggestionList(suggestions=[]) + + def _id_set_from_workflow(self, workflow_json: Dict[str, Any]): + """ + Return the sets of operator IDs and link IDs that already exist + in the workflow (after .model_dump()). + + * operators – dict mapping id → details + * links – list of link objects, each MAY contain linkID + """ + # Operators: keys of the dict + ops = set(workflow_json.get("operators", {}).keys()) + + # Links: collect linkID if present (model_dump() may omit it) + links = set() + for link in workflow_json.get("links", []): + if isinstance(link, dict): + lid = link.get("linkID") + if lid: # skip if missing / None + links.add(lid) + + return ops, links + + def _valid_suggestion_structure(self, s: Dict[str, Any]) -> bool: + """Quick structural check before deeper validation.""" + return ( + isinstance(s.get("suggestion"), str) + and s.get("suggestionType") in {"fix", "improve"} + and isinstance(s.get("changes"), dict) + and all( + k in s["changes"] + for k in ( + "operatorsToAdd", + "linksToAdd", + "operatorsToDelete", + "linksToDelete", + ) + ) + ) + + def _sanitize_suggestions( + self, + suggestions: List[Dict[str, Any]], + workflow_intp: BaseInterpretation, + ) -> SuggestionList: + """ + Validate / repair suggestions according to the project rules. + Invalid suggestions are dropped. In addition, port IDs in + `linksToAdd` are normalised to the Texera conventions: + • sources → "output-" (default index = 0) + • targets → "input-" (default index = 0) + """ + + # ---------- helper -------------------------------------------------- + def _normalise_port_id(raw_pid: Any, direction: str) -> str: + """ + Coerce various user/LLM styles into the canonical form. + + direction ∈ {"input", "output"} + """ + if raw_pid is None: + return f"{direction}-0" + + pid = str(raw_pid).strip().lower() + + # already canonical + if pid.startswith(f"{direction}-"): + return pid + + # just a bare integer (e.g. "0") + if pid.isdigit(): + return f"{direction}-{pid}" + + # naked "input"/"output" → assume 0 + if pid in {"input", "output"}: + return f"{direction}-0" + + # anything else – fall back to 0 + return f"{direction}-0" + + # -------------------------------------------------------------------- + + try: + workflow_dict = ( + workflow_intp.get_base_workflow_interpretation().model_dump() + ) + + existing_ops, existing_links = self._id_set_from_workflow(workflow_dict) + cleaned: List[Suggestion] = [] + operator_types_cache: Dict[str, Any] = {} + + for raw in suggestions: + # 0) basic shape & enums ------------------------------------------------- + if not self._valid_suggestion_structure(raw): + continue + + ch = raw["changes"] + + # 1) operatorsToAdd / updates ------------------------------------------ + for op in ch["operatorsToAdd"]: + # schema check (and cache) + optype = op["operatorType"] + if optype not in operator_types_cache: + try: + operator_types_cache[optype] = extract_json_schemas( + [optype] + )[0] + except ValueError: + break # invalid operator type -> drop suggestion + + # if updating an existing op, ID must exist in workflow + # (otherwise it's a brand-new op, which is fine) + else: + # 2) linksToAdd ------------------------------------------------------ + valid_link_add = True + for link in ch["linksToAdd"]: + # ---------- NEW: fix / fill port IDs -------------------------- + # NB: we *mutate* the dict in-place so later checks see the + # corrected value. + src_port = link.setdefault("source", {}) + tgt_port = link.setdefault("target", {}) + src_port["portID"] = _normalise_port_id( + src_port.get("portID"), direction="output" + ) + tgt_port["portID"] = _normalise_port_id( + tgt_port.get("portID"), direction="input" + ) + # -------------------------------------------------------------- + + # ensure linkID exists (generate if missing) + if not link.get("linkID"): + link["linkID"] = f"link-{uuid.uuid4()}" + + # ensure endpoints refer to real or newly-added ops + src_ok = src_port["operatorID"] in existing_ops or any( + op["operatorID"] == src_port["operatorID"] + for op in ch["operatorsToAdd"] + ) + tgt_ok = tgt_port["operatorID"] in existing_ops or any( + op["operatorID"] == tgt_port["operatorID"] + for op in ch["operatorsToAdd"] + ) + if not (src_ok and tgt_ok): + valid_link_add = False + break + if not valid_link_add: + continue # invalid suggestion + + # 3) deletions ------------------------------------------------------ + if not set(ch["operatorsToDelete"]).issubset(existing_ops): + continue + if not set(ch["linksToDelete"]).issubset(existing_links): + continue + + # 4) final parse to pydantic --------------------------------------- + try: + cleaned.append(Suggestion(**raw)) + except Exception: + continue # any remaining schema errors – drop + + return SuggestionList(suggestions=cleaned) + + except Exception as e: + print(f"Error sanitizing suggestions: {e}") + return SuggestionList(suggestions=[]) + + except Exception as e: + print(f"Error sanitizing suggestions: {e}") + return SuggestionList(suggestions=[]) + + def _call_function(self, name, args): + if name == "extract_json_schemas": + return extract_json_schemas(**args) + else: + raise ValueError(f"Unknown function: {name}") + + def _generate_suggestions_with_function_calls( + self, prompt: SuggestionPrompt, temperature: float, max_tokens: Optional[int] + ) -> SuggestionList: + """ + Two‑step approach: + 1) Ask the model which operator types it needs & have it call + `extract_json_schemas` (one call per distinct type). + 2) Execute those calls locally, hand results back, let the model + produce the final SuggestionList JSON. + """ + prompt_json: str = prompt.model_dump_json(indent=2) + input_messages = [{"role": "user", "content": prompt_json}] + # -- 1️⃣ first turn: let the model issue tool calls ------------------ + first_resp = self.client.responses.create( + model=self.model, + instructions=self.instruction_for_suggestion_fc, + input=input_messages, + tool_choice="required", + tools=self.function_tools, + ) + + for tool_call in first_resp.output: + if tool_call.type != "function_call": + continue + + name = tool_call.name + args = json.loads(tool_call.arguments) + + result = self._call_function(name, args) + input_messages.append( + { + "type": "function_call", + "call_id": tool_call.call_id, + "name": tool_call.name, + "arguments": tool_call.arguments, + } + ) + input_messages.append( + { + "type": "function_call_output", + "call_id": tool_call.call_id, + "output": str(result), + } + ) + + # -- 3️⃣ second turn: give the model the schemas & ask for suggestions + second_resp = self.client.responses.create( + model=self.model, + input=input_messages, + instructions=self.instruction_for_suggestion_fc, + tools=self.function_tools, + tool_choice="none", + text={ + "format": { # enforce SuggestionList schema again + "type": "json_schema", + "name": "workflow_suggestions", + "schema": SuggestionList.model_json_schema(), + "strict": False, + } + }, + ) + raw_suggestions = json.loads(second_resp.output_text)["suggestions"] + with open("raw_suggestions_debug.json", "w") as f: + json.dump(raw_suggestions, f, indent=2) + + sanitized_suggestions = self._sanitize_suggestions( + raw_suggestions, prompt.workflowInterpretation + ) + + return sanitized_suggestions + + def generate_data_cleaning_suggestions( + self, + prompt: DataCleaningSuggestionPrompt, + temperature: float = 0.7, + max_tokens: Optional[int] = None, + **kwargs, + ) -> DataCleaningSuggestionList: + """ + Produce a short list of natural-language cleaning recommendations + for the given column profile. + + The model is instructed to return a JSON object that matches + `DataCleaningSuggestionList` (see model.llm.suggestion). + """ + + try: + prompt_json = prompt.model_dump_json(indent=2) + + resp = self.client.responses.create( + model=self.model, + instructions=self.instruction_for_dc_suggestion, + input=prompt_json, + text={ + "format": { + "type": "json_schema", + "name": "data_cleaning_suggestions", + "schema": DataCleaningSuggestionList.model_json_schema(), + "strict": True, # let the model vary order / whitespace + "additionalProperties": False, + } + }, + ) + + # The assistant output is guaranteed JSON thanks to the schema- + # enforcement, so we can safely load then validate with pydantic. + suggestions_dict = json.loads(resp.output_text) + return suggestions_dict + + except Exception as e: + # Any error → return an empty list to the caller + print(f"[OpenAIAgent] Error in data-cleaning generation: {e}") + return DataCleaningSuggestionList(suggestions=[]) diff --git a/core/suggestion-service/llm_agent/utils/__init__.py b/core/suggestion-service/llm_agent/utils/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/llm_agent/utils/operator_metadata_converter.py b/core/suggestion-service/llm_agent/utils/operator_metadata_converter.py new file mode 100644 index 00000000000..e92c6ad5680 --- /dev/null +++ b/core/suggestion-service/llm_agent/utils/operator_metadata_converter.py @@ -0,0 +1,171 @@ +import json +import uuid +from typing import Dict, Any, List +import copy + +# Load the operator metadata file +with open("files/operator_json_schema.json", "r") as f: + operator_metadata = json.load(f) + +valid_operator_types: set[str] = { + schema["operatorType"] for schema in operator_metadata["operators"] +} + + +def resolve_ref(ref: str, definitions: Dict[str, Any]) -> Dict[str, Any]: + try: + ref_path = ref.lstrip("#/").split("/") + resolved = definitions + for part in ref_path: + resolved = resolved.get(part, {}) + return resolved + except Exception: + return {} + + +def fill_defaults( + schema: Dict[str, Any], definitions: Dict[str, Any] +) -> Dict[str, Any]: + """ + Fills default values into a schema-defined object. + """ + if "$ref" in schema: + schema = resolve_ref(schema["$ref"], definitions) + + if schema.get("type") == "object": + obj = {} + for prop, subschema in schema.get("properties", {}).items(): + obj[prop] = fill_defaults(subschema, definitions) + return obj + elif schema.get("type") == "array": + return [] + elif "default" in schema: + return copy.deepcopy(schema["default"]) + return None + + +def convert_to_operator_predicate(schema: Dict[str, Any]) -> Dict[str, Any]: + operator_type = schema["operatorType"] + metadata = schema["additionalMetadata"] + json_schema = schema["jsonSchema"] + definitions = json_schema.get("definitions", {}) + + operator_id = f"{operator_type}-operator-{uuid.uuid4()}" + operator_properties = fill_defaults(json_schema, definitions) + + input_ports = [] + for i, port_info in enumerate(metadata.get("inputPorts", [])): + input_ports.append( + { + "portID": f"input-{i}", + "displayName": port_info.get("displayName", ""), + "allowMultiInputs": port_info.get("allowMultiLinks", False), + "isDynamicPort": False, + "dependencies": port_info.get("dependencies", []), + } + ) + + output_ports = [] + for i, port_info in enumerate(metadata.get("outputPorts", [])): + output_ports.append( + { + "portID": f"output-{i}", + "displayName": port_info.get("displayName", ""), + "allowMultiInputs": False, + "isDynamicPort": False, + } + ) + + return { + "operatorID": operator_id, + "operatorType": operator_type, + "operatorVersion": schema.get("operatorVersion", "N/A"), + "operatorProperties": operator_properties, + "inputPorts": input_ports, + "outputPorts": output_ports, + "dynamicInputPorts": metadata.get("dynamicInputPorts", False), + "dynamicOutputPorts": metadata.get("dynamicOutputPorts", False), + "showAdvanced": False, + "isDisabled": False, + "customDisplayName": metadata.get("userFriendlyName", operator_type), + } + + +def _extract_json_schema( + operator_type: str, properties_only: bool = True +) -> Dict[str, Any]: + """ + Extract the full or properties-only JSON schema of an operatorType. + + Args: + operator_type: The operatorType string to search for + properties_only: If True, only include "properties", "definitions", "required" fields + and remove dummy/partition-related parts + + Returns: + A dictionary containing the extracted JSON schema. + """ + if operator_type not in valid_operator_types: + raise ValueError(f"OperatorType '{operator_type}' not found in metadata.") + + for operator_schema in operator_metadata["operators"]: + if operator_schema["operatorType"] == operator_type: + json_schema = operator_schema.get("jsonSchema") + if not json_schema: + raise ValueError( + f"No jsonSchema found for operatorType '{operator_type}'" + ) + + if not properties_only: + return {"operatorType": operator_type, "jsonSchema": json_schema} + + # Deepcopy to avoid modifying the original schema + filtered_schema = { + "properties": {}, + "definitions": {}, + "required": json_schema.get("required", []), + } + + for prop_name, prop_value in json_schema.get("properties", {}).items(): + if prop_name != "dummyPropertyList": + filtered_schema["properties"][prop_name] = prop_value + + for def_name, def_value in json_schema.get("definitions", {}).items(): + if def_name != "DummyProperties" and not def_name.endswith("Partition"): + filtered_schema["definitions"][def_name] = def_value + + return {"operatorType": operator_type, "jsonSchema": filtered_schema} + + raise ValueError(f"OperatorType '{operator_type}' not found unexpectedly.") + + +if __name__ == "__main__": + # Convert all schemas + operator_predicates: List[Dict[str, Any]] = [ + convert_to_operator_predicate(schema) + for schema in operator_metadata["operators"] + ] + output_path = "files/operator_format.json" + with open(output_path, "w") as f: + json.dump(operator_predicates, f, indent=2) + + operator_type_filepath = "files/operator_type.txt" + with open(operator_type_filepath, "w") as f: + f.write(str(list(valid_operator_types))) + + md_path = "files/operator_type_and_descriptions.md" + with open(md_path, "w") as f: + # optional title – remove if you don't want a header + f.write("# Operator Types and Descriptions\n\n") + + for op in operator_metadata["operators"]: + op_type = op["operatorType"] + op_desc = op.get("additionalMetadata", {}).get("operatorDescription", "") + # fallback if description is missing + if not op_desc: + op_desc = "(no description provided)" + f.write(f"- {op_type}: {op_desc}\n") + + +def extract_json_schemas(operator_types: List[str]) -> List[Dict[str, Any]]: + return [_extract_json_schema(op_type, True) for op_type in operator_types] diff --git a/core/suggestion-service/model/DataSchema.py b/core/suggestion-service/model/DataSchema.py new file mode 100644 index 00000000000..fa1ee8c13f5 --- /dev/null +++ b/core/suggestion-service/model/DataSchema.py @@ -0,0 +1,100 @@ +# the abstraction of data schema and attributes + +## define some constants related to type +### maybe define a class/enum called: AttributeType +### the possible values are: Integer, Long, Double, String, Boolean, binary + + +## the method of Attribute(not an abstract class), +## member variables: +## - name: the name of the attribute, a str +## - type: AttributeType +## methods: Getters for the name and type +## constructor: take the name and type to construct + + +## the method of DataSchema, which should also be a concrete class +## member variables: +## - a set of Attribute + +## methods: +## - the getter to return this set of attributes + +## constructors +## take a list of Attriubte +from enum import Enum +from typing import List, Set, Dict + +# Define the AttributeType enum +from enum import Enum +from typing import List, FrozenSet + + +# Define the AttributeType enum +class AttributeType(Enum): + INTEGER = "integer" + LONG = "long" + DOUBLE = "double" + STRING = "string" + BOOLEAN = "boolean" + BINARY = "binary" + TIMESTAMP = "timestamp" + + +# Define the Attribute class +class Attribute: + def __init__(self, name: str, attr_type: AttributeType): + self._name = name + self._type = attr_type + + @property + def name(self) -> str: + return self._name + + @property + def type(self) -> AttributeType: + return self._type + + def __str__(self) -> str: + return f"Attribute(name={self.name}, type={self.type.value})" + + def __repr__(self) -> str: + return f"Attribute(name={self.name!r}, type={self.type!r})" + + def __eq__(self, other) -> bool: + if isinstance(other, Attribute): + return self._name == other._name and self._type == other._type + return False + + def __hash__(self) -> int: + return hash((self._name, self._type)) + + +# Define the DataSchema class +class DataSchema: + def __init__(self, attributes: List[Dict]): + attributeList = [ + Attribute(a.get("attributeName"), a.get("attributeType")) + for a in attributes + ] + self._attributes: FrozenSet[Attribute] = frozenset(attributeList) + + @property + def attributes(self) -> FrozenSet[Attribute]: + return self._attributes + + def __str__(self) -> str: + attributes_str = ", ".join(str(attr) for attr in self.attributes) + return f"DataSchema(attributes=[{attributes_str}])" + + def __repr__(self) -> str: + attributes_repr = ", ".join(repr(attr) for attr in self.attributes) + return f"DataSchema(attributes=[{attributes_repr}])" + + def __eq__(self, other) -> bool: + if isinstance(other, DataSchema): + return self._attributes == other._attributes + return False + + def __hash__(self) -> int: + return hash(self._attributes) diff --git a/core/suggestion-service/model/EditingOperation.py b/core/suggestion-service/model/EditingOperation.py new file mode 100644 index 00000000000..597106c6647 --- /dev/null +++ b/core/suggestion-service/model/EditingOperation.py @@ -0,0 +1,81 @@ +from typing import Tuple, List + +from abc import ABC, abstractmethod + +from model.Operator import Operator +from model.Port import Port +from model.Workflow import Workflow + + +class EditingOperationType: + AddOperator = "add_op" + RemoveOperator = "remove_op" + UpdateOperator = "edit_op" + AddLink = "add_link" + RemoveLink = "remove_link" + UpdateLink = "edit_link" + Misc = "misc" + Unchanged = "unchanged" + Void = "void" + + +class EditingOperation(ABC): + @abstractmethod + def GetBaseWorkflow(self) -> Workflow: + pass + + @abstractmethod + def GetBase( + self, + ) -> ( + None + | List[Operator] + | List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + ): + """ + Return the base of this operation + add operator(s) -> None + remove operator(s) -> List[Operator] + modify operator(s) -> List[Operator] + + add link -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + remove link -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + modify link -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + """ + pass + + @abstractmethod + def GetModification( + self, + ) -> ( + None + | List[Operator] + | List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + ): + """ + Return "what's new" brought by this patch. + + add operator(s) -> List[Operator] + remove operator(s) -> None + modify operator(s) -> List[Operator] + + add link -> None + remove link -> None + modify link -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]] + """ + pass + + @abstractmethod + def GetType(self) -> EditingOperationType: + """ + Return the type of the operation + """ + pass + + @abstractmethod + def GetRawPatch(self) -> dict: + pass + + @abstractmethod + def IsValid(self) -> bool: + pass diff --git a/core/suggestion-service/model/Operator.py b/core/suggestion-service/model/Operator.py new file mode 100644 index 00000000000..e4ebcc157e2 --- /dev/null +++ b/core/suggestion-service/model/Operator.py @@ -0,0 +1,103 @@ +# the abstraction of operators + +# Operator Methods +# - GetName() -> str, return the name of the operator(e.g. Filter out all qualtified year) +# - GetType() -> str, return the type of the operator(e.g. FileScan, CSVFileScan, Filter, Projection) +# - GetId() -> str, return the id of the operator +# - GetProperties() -> dict, return a dict that contains the key(property name), the value(property values, could be single value, tuple, another dict, or list) +# - GetInputPort() -> list of Ports +# - GetOutputPort() -> list of Ports +# - GetInputSchema() -> dict, the key is the input port, the value is the DataSchema +# - GetOutputSchema() -> dict, the key is the output port, the value is the DataSchema +# - IsDynamicInputPorts() -> bool +# - IsDynamicOutputPorts() -> bool +# - IsDisabled() -> bool +# - IsViewResult() -> bool + +from abc import ABC, abstractmethod +from typing import Dict, List + + +class Operator(ABC): + @abstractmethod + def GetName(self) -> str: + """ + Return the name of the operator (e.g. 'Filter out all qualified year'). + """ + pass + + @abstractmethod + def GetType(self) -> str: + """ + Return the type of the operator (e.g. 'FileScan', 'CSVFileScan', 'Filter', 'Projection'). + """ + pass + + @abstractmethod + def GetId(self) -> str: + """ + Return the ID of the operator. + """ + pass + + @abstractmethod + def GetProperties(self) -> Dict: + """ + Return a dictionary containing the properties of the operator. + The dictionary can contain property names as keys and property values (which could be a single value, tuple, another dict, or list) as values. + """ + pass + + @abstractmethod + def GetInputSchemaByPortID(self, portID: str) -> "DataSchema": + pass + + @abstractmethod + def GetInputPorts(self) -> List["Port"]: + """ + Return a list of input ports for the operator. + """ + pass + + @abstractmethod + def GetOutputPorts(self) -> List["Port"]: + """ + Return a list of output ports for the operator. + """ + pass + + @abstractmethod + def GetError(self) -> str: + """ + Return the static error on this operator if any + :return: + """ + pass + + @abstractmethod + def IsDynamicInputPorts(self) -> bool: + """ + Return whether the operator has dynamic input ports. + """ + pass + + @abstractmethod + def IsDynamicOutputPorts(self) -> bool: + """ + Return whether the operator has dynamic output ports. + """ + pass + + @abstractmethod + def IsDisabled(self) -> bool: + """ + Return whether the operator is disabled. + """ + pass + + @abstractmethod + def IsViewResult(self) -> bool: + """ + Return whether the operator is for viewing results. + """ + pass diff --git a/core/suggestion-service/model/Port.py b/core/suggestion-service/model/Port.py new file mode 100644 index 00000000000..2d2ac14a18e --- /dev/null +++ b/core/suggestion-service/model/Port.py @@ -0,0 +1,90 @@ +# the abstraction of port + +# Port Methods +# IsInputPort() -> bool, return whether this port is input port +# IsOutputPort() -> bool, return whether this port is output port +# GetId() -> str, return the id of the port +# GetDisplayName() -> str, return the display name of the port +# AllowMultiInputs() -> bool, return whether the port allow multiple input edges +# IsDynamicPort() -> bool, return whether this port is a dynamic port +# Dependencies() -> str[], return a list of dependencies of the port id str if any + +from abc import ABC, abstractmethod +from typing import List + + +class Port(ABC): + @abstractmethod + def IsInputPort(self) -> bool: + """ + Return whether this port is an input port. + """ + pass + + @abstractmethod + def IsOutputPort(self) -> bool: + """ + Return whether this port is an output port. + """ + pass + + @abstractmethod + def GetId(self) -> str: + """ + Return the ID of the port. + """ + pass + + @abstractmethod + def GetDisplayName(self) -> str: + """ + Return the display name of the port. + """ + pass + + @abstractmethod + def AllowMultiInputs(self) -> bool: + """ + Return whether the port allows multiple input edges. + """ + pass + + @abstractmethod + def IsDynamicPort(self) -> bool: + """ + Return whether this port is a dynamic port. + """ + pass + + @abstractmethod + def GetDataSchema(self) -> "DataSchema": + """ + return the data schema annotated to this port + """ + pass + + @abstractmethod + def GetDependencies(self) -> List[str]: + """ + Return a list of dependencies of the port ID as strings, if any. + """ + pass + + @abstractmethod + def GetAffiliateOperator(self) -> "Operator": + """ + Return the operator that this port is affiliated to + """ + pass + + def GetTargetPorts(self) -> List["Port"]: + """ + Return the list of ports that this port(must be an output port) is sourcing + """ + pass + + def GetSourcePorts(self) -> List["Port"]: + """ + Return the list of ports that this port(must be an input port) is being targeting + """ + pass diff --git a/core/suggestion-service/model/Tuple.py b/core/suggestion-service/model/Tuple.py new file mode 100644 index 00000000000..4a53518269b --- /dev/null +++ b/core/suggestion-service/model/Tuple.py @@ -0,0 +1,169 @@ +from typing import Dict, List, Any, Optional, OrderedDict, Iterator, Union + +from model.DataSchema import DataSchema, Attribute + + +class Tuple: + """ + A simple implementation of a tuple for data storage. + """ + + def __init__( + self, data: Optional[Dict[str, Any]] = None, schema: Optional[DataSchema] = None + ): + """ + Initialize a Tuple with given data and optional schema. + + :param data: Dictionary mapping field names to values + :param schema: Optional schema that defines the structure of this tuple + """ + self._data: Dict[str, Any] = data or {} + self._schema = schema + + def __getitem__(self, key: Union[str, int]) -> Any: + """ + Get a field value by name or index. + + :param key: Field name (str) or index (int) + :return: Value of the field + """ + if isinstance(key, int): + key = list(self._data.keys())[key] + return self._data.get(key) + + def __setitem__(self, key: str, value: Any) -> None: + """ + Set a field value. + + :param key: Field name + :param value: Field value + """ + self._data[key] = value + + def __contains__(self, key: str) -> bool: + """ + Check if the tuple contains a field. + + :param key: Field name + :return: True if the field exists in the tuple + """ + return key in self._data + + def __len__(self) -> int: + """ + Get the number of fields in the tuple. + + :return: Number of fields + """ + return len(self._data) + + def __iter__(self) -> Iterator[Any]: + """ + Iterate over field values. + + :return: Iterator over field values + """ + return iter(self._data.values()) + + def __str__(self) -> str: + """ + String representation of the tuple. + + :return: String representation + """ + return f"Tuple({self._data})" + + def __repr__(self) -> str: + return self.__str__() + + def __eq__(self, other: object) -> bool: + """ + Check equality with another tuple. + + :param other: Another tuple + :return: True if tuples are equal + """ + if not isinstance(other, Tuple): + return False + return self._data == other._data + + def get_field_names(self) -> List[str]: + """ + Get list of field names in the tuple. + + :return: List of field names + """ + return list(self._data.keys()) + + def get_fields(self) -> Dict[str, Any]: + """ + Get dictionary representation of the tuple. + + :return: Dictionary with field names as keys and values as values + """ + return self._data.copy() + + def get_field(self, field_name: str) -> Any: + """ + Get value of a specific field. + + :param field_name: Name of the field + :return: Value of the field + """ + return self._data.get(field_name) + + def set_field(self, field_name: str, value: Any) -> None: + """ + Set value of a specific field. + + :param field_name: Name of the field + :param value: Value to set + """ + self._data[field_name] = value + + def as_dict(self) -> Dict[str, Any]: + """ + Convert tuple to dictionary. + + :return: Dictionary representation of the tuple + """ + return self._data.copy() + + def set_schema(self, schema: DataSchema) -> None: + """ + Set schema for the tuple. + + :param schema: Schema to set + """ + self._schema = schema + + def get_schema(self) -> Optional[DataSchema]: + """ + Get schema of the tuple. + + :return: Schema of the tuple + """ + return self._schema + + @classmethod + def from_dict(cls, data: Dict[str, Any]) -> "Tuple": + """ + Create a tuple from a dictionary. + + :param data: Dictionary with field names and values + :return: New Tuple instance + """ + return cls(data) + + @classmethod + def from_list(cls, field_names: List[str], values: List[Any]) -> "Tuple": + """ + Create a tuple from lists of field names and values. + + :param field_names: List of field names + :param values: List of values + :return: New Tuple instance + """ + if len(field_names) != len(values): + raise ValueError("Number of field names must match number of values") + return cls(dict(zip(field_names, values))) diff --git a/core/suggestion-service/model/Workflow.py b/core/suggestion-service/model/Workflow.py new file mode 100644 index 00000000000..6a8656192ed --- /dev/null +++ b/core/suggestion-service/model/Workflow.py @@ -0,0 +1,72 @@ +# the abstraction of workflows +# Methods +# 1. GetOperators(a list of operator type) -> a list of operators, if given list of types are none empty, filter and only keep those operators that of those types +# 2. GetLinks() -> a list of links(between these operators) +# 3. GetSubWorkflowByIndex(idx: int) -> (Workflow, one operator, a list of link related to that operator): this method will topologically sort the operators in the workflow, and +# it will return a sub workflow containing operator [0, idx-1], the idx-th operator, the links connected to this idx-th operator. + + +from abc import ABC, abstractmethod +from typing import List, Tuple, Dict + +from model.Operator import Operator +from model.Port import Port + + +class Workflow(ABC): + @abstractmethod + def GetWorkflowContent(self) -> str: + """ + return the raw content of the workflow + :return: + """ + pass + + @abstractmethod + def GetWorkflowId(self) -> int: + """ + return the id of the workflow + :return: + """ + pass + + @abstractmethod + def GetOperators(self, types: List[str] = None) -> List["Operator"]: + """ + Return a list of operators. If the given list of types is non-empty, + filter and only keep those operators of those types. + """ + pass + + @abstractmethod + def TopologicalSort(self) -> List["Operator"]: + """ + Perform a topological sort on the operators in the workflow. + """ + pass + + @abstractmethod + def VisualizeDAG(self): + pass + + @abstractmethod + def GetDAG(self): + pass + + @abstractmethod + def GetSchemaToNextOperatorDistributionMapping( + self, + ) -> Dict["DataSchema", Dict[str, int]]: + pass + + @abstractmethod + def GetOperatorTypeToNextOperatorDistributionMapping( + self, + ) -> Dict[str, Dict[str, int]]: + pass + + @abstractmethod + def GetAdditionPairs( + self, + ) -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]]: + pass diff --git a/core/suggestion-service/model/__init__.py b/core/suggestion-service/model/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/llm/__init__.py b/core/suggestion-service/model/llm/__init__.py new file mode 100644 index 00000000000..3df30d7ba22 --- /dev/null +++ b/core/suggestion-service/model/llm/__init__.py @@ -0,0 +1 @@ +from .prompt import SuggestionPrompt # noqa: F401 diff --git a/core/suggestion-service/model/llm/interpretation.py b/core/suggestion-service/model/llm/interpretation.py new file mode 100644 index 00000000000..37065aacc2a --- /dev/null +++ b/core/suggestion-service/model/llm/interpretation.py @@ -0,0 +1,89 @@ +# model/serialization.py +from abc import ABC, abstractmethod +from enum import Enum + +from pydantic import BaseModel, Field +from typing import Dict, List, Optional, Any + + +class AttributeInterpretation(BaseModel): + attributeName: str + attributeType: str + + +class SchemaInterpretation(BaseModel): + attributes: List[AttributeInterpretation] + + +class PortInterpretation(BaseModel): + portID: str + inputSchema: SchemaInterpretation + + +class ErrorInterpretation(BaseModel): + type: str + message: str + details: str + + +class OperatorInterpretation(BaseModel): + operatorID: str + operatorType: str + customDisplayName: Optional[str] + operatorProperties: Dict[str, Any] + error: Optional[ErrorInterpretation] + inputSchemas: Dict[str, PortInterpretation] + outputSchema: SchemaInterpretation = Field( + default_factory=lambda: SchemaInterpretation(attributes=[]) + ) + + +class LinkEndInterpretation(BaseModel): + operatorID: str + portID: str + + +class LinkInterpretation(BaseModel): + linkID: str + source: LinkEndInterpretation + target: LinkEndInterpretation + + +class WorkflowInterpretation(BaseModel): + operators: Dict[str, OperatorInterpretation] + links: List[LinkInterpretation] + + +# Base interpretation class for commonality among interpretations +class BaseInterpretation(BaseModel, ABC): + """Abstract base class for workflow interpretation variants.""" + + class Config: + extra = "forbid" + + @abstractmethod + def get_base_workflow_interpretation(self) -> WorkflowInterpretation: + """Returns one or more workflow interpretations (typically just one for Raw, or multiple for Path).""" + pass + + +# Interpretation derived from linear execution paths +class PathInterpretation(BaseInterpretation): + workflow: WorkflowInterpretation + paths: List[List[str]] + + def get_base_workflow_interpretation(self) -> WorkflowInterpretation: + return self.workflow + + +# Interpretation derived directly from the raw workflow object +class RawInterpretation(BaseInterpretation): + workflow: WorkflowInterpretation + + def get_base_workflow_interpretation(self) -> WorkflowInterpretation: + return self.workflow + + +class InterpretationMethod(Enum): + RAW = "raw" + BY_PATH = "by_path" diff --git a/core/suggestion-service/model/llm/prompt.py b/core/suggestion-service/model/llm/prompt.py new file mode 100644 index 00000000000..0933504ba45 --- /dev/null +++ b/core/suggestion-service/model/llm/prompt.py @@ -0,0 +1,52 @@ +from pydantic import BaseModel, Field +from typing import List, Union + +from model.llm.interpretation import ( + OperatorInterpretation, + BaseInterpretation, + PathInterpretation, + RawInterpretation, + SchemaInterpretation, +) + +# Import all relevant classes from the proto definition +from model.proto.edu.uci.ics.amber.engine.architecture.worker import ( + TableProfile, + GlobalProfile, + ColumnProfile, + NumericMatrix, + ColumnIndexList, + GlobalProfileTimes, + ColumnStatistics, +) + + +class SuggestionPrompt(BaseModel): + """Prompt structure sent to the LLM agent. + + It combines the user intention, the operators that the user wishes to focus on, + and the interpretation of the whole workflow. + """ + + # A short sentence or question describing what the user wants from the LLM. + intention: str = Field( + "Recommend improvements and fixes of current workflows", + description="The user's intention for generating suggestions.", + ) + + # The subset of operators that the user explicitly wants the LLM to pay extra attention to. + focusingOperators: List[OperatorInterpretation] = Field( + default_factory=list, + description="Operators that the user is focusing on.", + ) + + # Full interpretation of the workflow (either PathInterpretation or RawInterpretation) + workflowInterpretation: Union[PathInterpretation, RawInterpretation] = Field( + ..., description="Interpretation of the complete workflow." + ) + + +class DataCleaningSuggestionPrompt(BaseModel): + focusingOperatorID: str + columnProfile: ColumnProfile + tableSchema: SchemaInterpretation diff --git a/core/suggestion-service/model/llm/sanitizor.py b/core/suggestion-service/model/llm/sanitizor.py new file mode 100644 index 00000000000..b2a710da37c --- /dev/null +++ b/core/suggestion-service/model/llm/sanitizor.py @@ -0,0 +1,14 @@ +from pydantic import BaseModel +from typing import Dict, List, Optional, Any + +from model.llm.suggestion import Suggestion, SuggestionList + + +class OperatorSchema(BaseModel): + operatorType: str + jsonSchema: dict + + +class SuggestionSanitization(BaseModel): + suggestions: SuggestionList + schemas: List[OperatorSchema] diff --git a/core/suggestion-service/model/llm/suggestion.py b/core/suggestion-service/model/llm/suggestion.py new file mode 100644 index 00000000000..f5c8883e496 --- /dev/null +++ b/core/suggestion-service/model/llm/suggestion.py @@ -0,0 +1,81 @@ +from pydantic import BaseModel, Field +from typing import List, Optional, Literal + + +class Operator(BaseModel): + operatorType: str + operatorID: str + operatorProperties: dict + customDisplayName: Optional[str] = None + + class Config: + json_schema_extra = { + "required": ["operatorType", "operatorID", "operatorProperties"] + } + + +class Port(BaseModel): + operatorID: str + portID: str + + class Config: + json_schema_extra = {"required": ["operatorID", "portID"]} + + +class Link(BaseModel): + linkID: str + source: Port + target: Port + + class Config: + json_schema_extra = {"required": ["linkID", "source", "target"]} + + +class Changes(BaseModel): + operatorsToAdd: List[Operator] + linksToAdd: List[Link] + operatorsToDelete: List[str] + linksToDelete: List[str] + + class Config: + json_schema_extra = { + "required": [ + "operatorsToAdd", + "linksToAdd", + "operatorsToDelete", + "linksToDelete", + ] + } + + +class Suggestion(BaseModel): + suggestion: str + suggestionType: Literal["fix", "improve"] + changes: Changes + + class Config: + json_schema_extra = {"required": ["suggestion", "suggestionType", "changes"]} + + +class SuggestionList(BaseModel): + suggestions: List[Suggestion] + + class Config: + json_schema_extra = {"required": ["suggestions"]} + + +class DataCleaningSuggestion(BaseModel): + suggestion: str + details: str + + class Config: + json_schema_extra = {"required": ["suggestion", "details"]} + extra = "forbid" + + +class DataCleaningSuggestionList(BaseModel): + suggestions: List[DataCleaningSuggestion] + + class Config: + json_schema_extra = {"required": ["suggestions"]} + extra = "forbid" diff --git a/core/suggestion-service/model/proto/__init__.py b/core/suggestion-service/model/proto/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/__init__.py b/core/suggestion-service/model/proto/edu/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/__init__.py b/core/suggestion-service/model/proto/edu/uci/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/ics/__init__.py b/core/suggestion-service/model/proto/edu/uci/ics/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/ics/amber/__init__.py b/core/suggestion-service/model/proto/edu/uci/ics/amber/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/__init__.py b/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/architecture/__init__.py b/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/architecture/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py b/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py new file mode 100644 index 00000000000..c9f130addda --- /dev/null +++ b/core/suggestion-service/model/proto/edu/uci/ics/amber/engine/architecture/worker/__init__.py @@ -0,0 +1,124 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# sources: edu/uci/ics/amber/engine/architecture/worker/tableprofile.proto +# plugin: python-betterproto +# This file has been @generated + +from dataclasses import dataclass +from typing import ( + Dict, + List, +) + +import betterproto + + +@dataclass(eq=False, repr=False) +class NumericMatrix(betterproto.Message): + values: List[float] = betterproto.double_field(1) + rows: int = betterproto.uint32_field(2) + cols: int = betterproto.uint32_field(3) + + +@dataclass(eq=False, repr=False) +class GlobalProfile(betterproto.Message): + samples_used: int = betterproto.uint64_field(1) + """---- basic counts ----""" + + column_count: int = betterproto.uint64_field(2) + row_count: int = betterproto.uint64_field(3) + row_has_null_ratio: float = betterproto.double_field(4) + """---- row null / uniqueness ----""" + + row_is_null_ratio: float = betterproto.double_field(5) + unique_row_ratio: float = betterproto.double_field(6) + duplicate_row_count: int = betterproto.uint64_field(7) + file_type: str = betterproto.string_field(8) + """---- metadata ----""" + + encoding: str = betterproto.string_field(9) + correlation_matrix: "NumericMatrix" = betterproto.message_field(10) + """---- pairwise stats ----""" + + chi2_matrix: "NumericMatrix" = betterproto.message_field(11) + profile_schema: Dict[str, "ColumnIndexList"] = betterproto.map_field( + 12, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE + ) + """---- schema map: column-name -> indices (to mirror DataProfiler) ---""" + + times: "GlobalProfileTimes" = betterproto.message_field(13) + + +@dataclass(eq=False, repr=False) +class GlobalProfileTimes(betterproto.Message): + """---- timing ----""" + + row_stats_ms: float = betterproto.double_field(1) + + +@dataclass(eq=False, repr=False) +class ColumnIndexList(betterproto.Message): + """helper for profile_schema""" + + indices: List[int] = betterproto.uint32_field(1) + + +@dataclass(eq=False, repr=False) +class ColumnStatistics(betterproto.Message): + min: float = betterproto.double_field(1) + """---- numeric summaries (nullable when not applicable) ----""" + + max: float = betterproto.double_field(2) + median: float = betterproto.double_field(3) + mean: float = betterproto.double_field(4) + variance: float = betterproto.double_field(5) + stddev: float = betterproto.double_field(6) + skewness: float = betterproto.double_field(7) + kurtosis: float = betterproto.double_field(8) + sum: float = betterproto.double_field(9) + quantiles: List[float] = betterproto.double_field(10) + """---- distribution ----""" + + num_zeros: int = betterproto.uint64_field(11) + num_negatives: int = betterproto.uint64_field(12) + unique_count: int = betterproto.uint64_field(13) + """---- uniqueness / cardinality ----""" + + unique_ratio: float = betterproto.double_field(14) + categorical: bool = betterproto.bool_field(15) + """---- categorical helpers ----""" + + categorical_count: Dict[str, int] = betterproto.map_field( + 16, betterproto.TYPE_STRING, betterproto.TYPE_UINT64 + ) + null_count: int = betterproto.uint64_field(17) + """---- nulls ----""" + + null_types: List[str] = betterproto.string_field(18) + data_type_representation: Dict[str, float] = betterproto.map_field( + 19, betterproto.TYPE_STRING, betterproto.TYPE_DOUBLE + ) + """---- data-type representation share (DataProfiler style) ----""" + + +@dataclass(eq=False, repr=False) +class ColumnProfile(betterproto.Message): + column_name: str = betterproto.string_field(1) + """identity""" + + data_type: str = betterproto.string_field(2) + data_label: str = betterproto.string_field(3) + categorical: bool = betterproto.bool_field(4) + """quick hints""" + + order: str = betterproto.string_field(5) + samples: List[str] = betterproto.string_field(6) + """examples""" + + statistics: "ColumnStatistics" = betterproto.message_field(7) + """heavy stats""" + + +@dataclass(eq=False, repr=False) +class TableProfile(betterproto.Message): + global_profile: "GlobalProfile" = betterproto.message_field(1) + column_profiles: List["ColumnProfile"] = betterproto.message_field(2) diff --git a/core/suggestion-service/model/proto/scalapb/__init__.py b/core/suggestion-service/model/proto/scalapb/__init__.py new file mode 100644 index 00000000000..49c713815a5 --- /dev/null +++ b/core/suggestion-service/model/proto/scalapb/__init__.py @@ -0,0 +1,421 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# sources: scalapb/scalapb.proto +# plugin: python-betterproto +# This file has been @generated + +from dataclasses import dataclass +from typing import ( + Dict, + List, +) + +import betterproto +import betterproto.lib.google.protobuf as betterproto_lib_google_protobuf + + +class MatchType(betterproto.Enum): + CONTAINS = 0 + EXACT = 1 + PRESENCE = 2 + + +class ScalaPbOptionsOptionsScope(betterproto.Enum): + """ + Whether to apply the options only to this file, or for the entire package (and its subpackages) + """ + + FILE = 0 + """Apply the options for this file only (default)""" + + PACKAGE = 1 + """Apply the options for the entire package and its subpackages.""" + + +class ScalaPbOptionsEnumValueNaming(betterproto.Enum): + """Naming convention for generated enum values""" + + AS_IN_PROTO = 0 + CAMEL_CASE = 1 + + +@dataclass(eq=False, repr=False) +class ScalaPbOptions(betterproto.Message): + package_name: str = betterproto.string_field(1) + """If set then it overrides the java_package and package.""" + + flat_package: bool = betterproto.bool_field(2) + """ + If true, the compiler does not append the proto base file name + into the generated package name. If false (the default), the + generated scala package name is the package_name.basename where + basename is the proto file name without the .proto extension. + """ + + import_: List[str] = betterproto.string_field(3) + """ + Adds the following imports at the top of the file (this is meant + to provide implicit TypeMappers) + """ + + preamble: List[str] = betterproto.string_field(4) + """ + Text to add to the generated scala file. This can be used only + when single_file is true. + """ + + single_file: bool = betterproto.bool_field(5) + """ + If true, all messages and enums (but not services) will be written + to a single Scala file. + """ + + no_primitive_wrappers: bool = betterproto.bool_field(7) + """ + By default, wrappers defined at + https://github.com/google/protobuf/blob/master/src/google/protobuf/wrappers.proto, + are mapped to an Option[T] where T is a primitive type. When this field + is set to true, we do not perform this transformation. + """ + + primitive_wrappers: bool = betterproto.bool_field(6) + """ + DEPRECATED. In ScalaPB <= 0.5.47, it was necessary to explicitly enable + primitive_wrappers. This field remains here for backwards compatibility, + but it has no effect on generated code. It is an error to set both + `primitive_wrappers` and `no_primitive_wrappers`. + """ + + collection_type: str = betterproto.string_field(8) + """ + Scala type to be used for repeated fields. If unspecified, + `scala.collection.Seq` will be used. + """ + + preserve_unknown_fields: bool = betterproto.bool_field(9) + """ + If set to true, all generated messages in this file will preserve unknown + fields. + """ + + object_name: str = betterproto.string_field(10) + """ + If defined, sets the name of the file-level object that would be generated. This + object extends `GeneratedFileObject` and contains descriptors, and list of message + and enum companions. + """ + + scope: "ScalaPbOptionsOptionsScope" = betterproto.enum_field(11) + """Experimental: scope to apply the given options.""" + + lenses: bool = betterproto.bool_field(12) + """If true, lenses will be generated.""" + + retain_source_code_info: bool = betterproto.bool_field(13) + """ + If true, then source-code info information will be included in the + generated code - normally the source code info is cleared out to reduce + code size. The source code info is useful for extracting source code + location from the descriptors as well as comments. + """ + + map_type: str = betterproto.string_field(14) + """ + Scala type to be used for maps. If unspecified, + `scala.collection.immutable.Map` will be used. + """ + + no_default_values_in_constructor: bool = betterproto.bool_field(15) + """ + If true, no default values will be generated in message constructors. + """ + + enum_value_naming: "ScalaPbOptionsEnumValueNaming" = betterproto.enum_field(16) + enum_strip_prefix: bool = betterproto.bool_field(17) + """ + Indicate if prefix (enum name + optional underscore) should be removed in scala code + Strip is applied before enum value naming changes. + """ + + bytes_type: str = betterproto.string_field(21) + """Scala type to use for bytes fields.""" + + java_conversions: bool = betterproto.bool_field(23) + """Enable java conversions for this file.""" + + aux_message_options: List["ScalaPbOptionsAuxMessageOptions"] = ( + betterproto.message_field(18) + ) + """List of message options to apply to some messages.""" + + aux_field_options: List["ScalaPbOptionsAuxFieldOptions"] = ( + betterproto.message_field(19) + ) + """List of message options to apply to some fields.""" + + aux_enum_options: List["ScalaPbOptionsAuxEnumOptions"] = betterproto.message_field( + 20 + ) + """List of message options to apply to some enums.""" + + aux_enum_value_options: List["ScalaPbOptionsAuxEnumValueOptions"] = ( + betterproto.message_field(22) + ) + """List of enum value options to apply to some enum values.""" + + preprocessors: List[str] = betterproto.string_field(24) + """List of preprocessors to apply.""" + + field_transformations: List["FieldTransformation"] = betterproto.message_field(25) + ignore_all_transformations: bool = betterproto.bool_field(26) + """ + Ignores all transformations for this file. This is meant to allow specific files to + opt out from transformations inherited through package-scoped options. + """ + + getters: bool = betterproto.bool_field(27) + """If true, getters will be generated.""" + + test_only_no_java_conversions: bool = betterproto.bool_field(999) + """ + For use in tests only. Inhibit Java conversions even when when generator parameters + request for it. + """ + + +@dataclass(eq=False, repr=False) +class ScalaPbOptionsAuxMessageOptions(betterproto.Message): + """ + AuxMessageOptions enables you to set message-level options through package-scoped options. + This is useful when you can't add a dependency on scalapb.proto from the proto file that + defines the message. + """ + + target: str = betterproto.string_field(1) + """The fully-qualified name of the message in the proto name space.""" + + options: "MessageOptions" = betterproto.message_field(2) + """ + Options to apply to the message. If there are any options defined on the target message + they take precedence over the options. + """ + + +@dataclass(eq=False, repr=False) +class ScalaPbOptionsAuxFieldOptions(betterproto.Message): + """ + AuxFieldOptions enables you to set field-level options through package-scoped options. + This is useful when you can't add a dependency on scalapb.proto from the proto file that + defines the field. + """ + + target: str = betterproto.string_field(1) + """The fully-qualified name of the field in the proto name space.""" + + options: "FieldOptions" = betterproto.message_field(2) + """ + Options to apply to the field. If there are any options defined on the target message + they take precedence over the options. + """ + + +@dataclass(eq=False, repr=False) +class ScalaPbOptionsAuxEnumOptions(betterproto.Message): + """ + AuxEnumOptions enables you to set enum-level options through package-scoped options. + This is useful when you can't add a dependency on scalapb.proto from the proto file that + defines the enum. + """ + + target: str = betterproto.string_field(1) + """The fully-qualified name of the enum in the proto name space.""" + + options: "EnumOptions" = betterproto.message_field(2) + """ + Options to apply to the enum. If there are any options defined on the target enum + they take precedence over the options. + """ + + +@dataclass(eq=False, repr=False) +class ScalaPbOptionsAuxEnumValueOptions(betterproto.Message): + """ + AuxEnumValueOptions enables you to set enum value level options through package-scoped + options. This is useful when you can't add a dependency on scalapb.proto from the proto + file that defines the enum. + """ + + target: str = betterproto.string_field(1) + """The fully-qualified name of the enum value in the proto name space.""" + + options: "EnumValueOptions" = betterproto.message_field(2) + """ + Options to apply to the enum value. If there are any options defined on + the target enum value they take precedence over the options. + """ + + +@dataclass(eq=False, repr=False) +class MessageOptions(betterproto.Message): + extends: List[str] = betterproto.string_field(1) + """Additional classes and traits to mix in to the case class.""" + + companion_extends: List[str] = betterproto.string_field(2) + """Additional classes and traits to mix in to the companion object.""" + + annotations: List[str] = betterproto.string_field(3) + """Custom annotations to add to the generated case class.""" + + type: str = betterproto.string_field(4) + """ + All instances of this message will be converted to this type. An implicit TypeMapper + must be present. + """ + + companion_annotations: List[str] = betterproto.string_field(5) + """ + Custom annotations to add to the companion object of the generated class. + """ + + sealed_oneof_extends: List[str] = betterproto.string_field(6) + """ + Additional classes and traits to mix in to generated sealed_oneof base trait. + """ + + no_box: bool = betterproto.bool_field(7) + """ + If true, when this message is used as an optional field, do not wrap it in an `Option`. + This is equivalent of setting `(field).no_box` to true on each field with the message type. + """ + + unknown_fields_annotations: List[str] = betterproto.string_field(8) + """ + Custom annotations to add to the generated `unknownFields` case class field. + """ + + +@dataclass(eq=False, repr=False) +class Collection(betterproto.Message): + """ + Represents a custom Collection type in Scala. This allows ScalaPB to integrate with + collection types that are different enough from the ones in the standard library. + """ + + type: str = betterproto.string_field(1) + """Type of the collection""" + + non_empty: bool = betterproto.bool_field(2) + """ + Set to true if this collection type is not allowed to be empty, for example + cats.data.NonEmptyList. When true, ScalaPB will not generate `clearX` for the repeated + field and not provide a default argument in the constructor. + """ + + adapter: str = betterproto.string_field(3) + """ + An Adapter is a Scala object available at runtime that provides certain static methods + that can operate on this collection type. + """ + + +@dataclass(eq=False, repr=False) +class FieldOptions(betterproto.Message): + type: str = betterproto.string_field(1) + scala_name: str = betterproto.string_field(2) + collection_type: str = betterproto.string_field(3) + """ + Can be specified only if this field is repeated. If unspecified, + it falls back to the file option named `collection_type`, which defaults + to `scala.collection.Seq`. + """ + + collection: "Collection" = betterproto.message_field(8) + key_type: str = betterproto.string_field(4) + """ + If the field is a map, you can specify custom Scala types for the key + or value. + """ + + value_type: str = betterproto.string_field(5) + annotations: List[str] = betterproto.string_field(6) + """Custom annotations to add to the field.""" + + map_type: str = betterproto.string_field(7) + """ + Can be specified only if this field is a map. If unspecified, + it falls back to the file option named `map_type` which defaults to + `scala.collection.immutable.Map` + """ + + no_box: bool = betterproto.bool_field(30) + """ + Do not box this value in Option[T]. If set, this overrides MessageOptions.no_box + """ + + required: bool = betterproto.bool_field(31) + """ + Like no_box it does not box a value in Option[T], but also fails parsing when a value + is not provided. This enables to emulate required fields in proto3. + """ + + +@dataclass(eq=False, repr=False) +class EnumOptions(betterproto.Message): + extends: List[str] = betterproto.string_field(1) + """Additional classes and traits to mix in to the base trait""" + + companion_extends: List[str] = betterproto.string_field(2) + """Additional classes and traits to mix in to the companion object.""" + + type: str = betterproto.string_field(3) + """ + All instances of this enum will be converted to this type. An implicit TypeMapper + must be present. + """ + + base_annotations: List[str] = betterproto.string_field(4) + """Custom annotations to add to the generated enum's base class.""" + + recognized_annotations: List[str] = betterproto.string_field(5) + """Custom annotations to add to the generated trait.""" + + unrecognized_annotations: List[str] = betterproto.string_field(6) + """Custom annotations to add to the generated Unrecognized case class.""" + + +@dataclass(eq=False, repr=False) +class EnumValueOptions(betterproto.Message): + extends: List[str] = betterproto.string_field(1) + """Additional classes and traits to mix in to an individual enum value.""" + + scala_name: str = betterproto.string_field(2) + """Name in Scala to use for this enum value.""" + + annotations: List[str] = betterproto.string_field(3) + """ + Custom annotations to add to the generated case object for this enum value. + """ + + +@dataclass(eq=False, repr=False) +class OneofOptions(betterproto.Message): + extends: List[str] = betterproto.string_field(1) + """Additional traits to mix in to a oneof.""" + + scala_name: str = betterproto.string_field(2) + """Name in Scala to use for this oneof field.""" + + +@dataclass(eq=False, repr=False) +class FieldTransformation(betterproto.Message): + when: "betterproto_lib_google_protobuf.FieldDescriptorProto" = ( + betterproto.message_field(1) + ) + match_type: "MatchType" = betterproto.enum_field(2) + set: "betterproto_lib_google_protobuf.FieldOptions" = betterproto.message_field(3) + + +@dataclass(eq=False, repr=False) +class PreprocessorOutput(betterproto.Message): + options_by_file: Dict[str, "ScalaPbOptions"] = betterproto.map_field( + 1, betterproto.TYPE_STRING, betterproto.TYPE_MESSAGE + ) diff --git a/core/suggestion-service/model/texera/TexeraOperator.py b/core/suggestion-service/model/texera/TexeraOperator.py new file mode 100644 index 00000000000..1dadffb1649 --- /dev/null +++ b/core/suggestion-service/model/texera/TexeraOperator.py @@ -0,0 +1,138 @@ +from typing import Dict, List, Tuple + +from model.DataSchema import DataSchema +from model.Operator import Operator +from model.llm.interpretation import PortInterpretation, OperatorInterpretation +from model.texera.TexeraPort import TexeraPort + + +class TexeraOperator(Operator): + def __init__( + self, + operator_dict: dict, + port_indexed_input_schemas: List["DataSchema"] = [], + error: dict = {}, + ): + self.operator_id = operator_dict.get("operatorID", "") + self.operator_type = operator_dict.get("operatorType", "") + self.operator_version = operator_dict.get("operatorVersion", "") + self.operator_properties = operator_dict.get("operatorProperties", {}) + + # a mapping from port id to port + self.input_ports: Dict[str, "TexeraPort"] = { + port_dict.get("portID"): TexeraPort( + port_dict=port_dict, + is_input_port=True, + operator=self, + schema=( + port_indexed_input_schemas[i] + if i < len(port_indexed_input_schemas) + else DataSchema([]) + ), + ) + for i, port_dict in enumerate(operator_dict.get("inputPorts", [])) + } + + self.output_ports: Dict[str, "TexeraPort"] = { + port_dict.get("portID"): TexeraPort( + port_dict=port_dict, + is_input_port=False, + operator=self, + schema=DataSchema([]), + ) + for port_dict in operator_dict.get("outputPorts", []) + } + self.show_advanced = operator_dict.get("showAdvanced", False) + self.is_disabled = operator_dict.get("isDisabled", False) + self.custom_display_name = operator_dict.get("customDisplayName", "") + self.dynamic_input_ports = operator_dict.get("dynamicInputPorts", False) + self.dynamic_output_ports = operator_dict.get("dynamicOutputPorts", False) + self.view_result = operator_dict.get("viewResult", False) + self.input_schema = port_indexed_input_schemas + if error != {}: + self.error = { + "type": error.get("type").get("name"), + "message": error.get("message"), + "details": error.get("details")[:200], + } + else: + self.error = None + + def GetName(self) -> str: + return self.custom_display_name + + def GetType(self) -> str: + return self.operator_type + + def GetId(self) -> str: + return self.operator_id + + def GetProperties(self) -> Dict: + return self.operator_properties + + def GetInputSchemaByPortID(self, portID: str) -> "DataSchema": + port = self.input_ports.get(portID) + if port is not None: + return port.GetDataSchema() + return None + + def GetInputPorts(self) -> List["Port"]: + return list(self.input_ports.values()) + + def GetOutputPorts(self) -> List["Port"]: + return list(self.output_ports.values()) + + def GetError(self) -> str: + return self.error + + def IsDynamicInputPorts(self) -> bool: + return self.dynamic_input_ports + + def IsDynamicOutputPorts(self) -> bool: + return self.dynamic_output_ports + + def IsDisabled(self) -> bool: + return self.is_disabled + + def IsViewResult(self) -> bool: + return self.view_result + + # Add this method to TexeraOperator + + def ToPydantic(self) -> OperatorInterpretation: + + return OperatorInterpretation( + operatorID=self.operator_id, + operatorType=self.operator_type, + customDisplayName=self.custom_display_name or None, + operatorProperties=self.operator_properties, + error=self.error, + inputSchemas={ + port.GetId(): port.ToPydantic() for port in self.GetInputPorts() + }, + ) + + def __str__(self) -> str: + input_ports_str = "\n ".join([str(port) for port in self.GetInputPorts()]) + output_ports_str = "\n ".join([str(port) for port in self.GetOutputPorts()]) + return ( + f"TexeraOperator(\n" + f" ID={self.operator_id},\n" + f" Type={self.operator_type},\n" + f" Version={self.operator_version},\n" + f" Properties={self.operator_properties},\n" + f" InputPorts=[\n {input_ports_str}\n ],\n" + f" OutputPorts=[\n {output_ports_str}\n ],\n" + f" Error={self.error}\n" + f")" + ) + + @property + def properties(self) -> Dict: + """ + Get the operator properties. + + Returns: + The operator properties dictionary + """ + return self.operator_properties diff --git a/core/suggestion-service/model/texera/TexeraOperatorTypes.py b/core/suggestion-service/model/texera/TexeraOperatorTypes.py new file mode 100644 index 00000000000..2175644829e --- /dev/null +++ b/core/suggestion-service/model/texera/TexeraOperatorTypes.py @@ -0,0 +1,77 @@ +class TexeraOperatorTypes: + class DataInput: + CSVFileScan = "CSVFileScan" + FileScan = "FileScan" + JSONLFileScan = "JSONLFileScan" + TextInput = "TextInput" + ParallelCSVFileScan = "ParallelCSVFileScan" + + class MachineLearning: + class SkLearn: + SklearnLogisticRegression = "SklearnLogisticRegression" + SklearnLogisticRegressionCV = "SklearnLogisticRegressionCV" + SklearnRidge = "SklearnRidge" + SklearnRidgeCV = "SklearnRidgeCV" + SklearnSDG = "SklearnSDG" + SklearnPassiveAggressive = "SklearnPassiveAggressive" + SklearnPerceptron = "SklearnPerceptron" + SklearnKNN = "SklearnKNN" + SklearnNearestCentroid = "SklearnNearestCentroid" + SklearnSVM = "SklearnSVM" + SklearnLinearSVM = "SklearnLinearSVM" + SklearnLinearRegression = "SklearnLinearRegression" + SklearnDecisionTree = "SklearnDecisionTree" + SklearnExtraTree = "SklearnExtraTree" + SklearnMultiLayerPerceptron = "SklearnMultiLayerPerceptron" + SklearnProbabilityCalibration = "SklearnProbabilityCalibration" + SklearnRandomForest = "SklearnRandomForest" + SklearnBagging = "SklearnBagging" + SklearnGradientBoosting = "SklearnGradientBoosting" + SklearnAdaptiveBoosting = "SklearnAdaptiveBoosting" + SklearnExtraTrees = "SklearnExtraTrees" + SklearnGaussianNaiveBayes = "SklearnGaussianNaiveBayes" + SklearnMultinomialNaiveBayes = "SklearnMultinomialNaiveBayes" + SklearnComplementNaiveBayes = "SklearnComplementNaiveBayes" + SklearnBernoulliNaiveBayes = "SklearnBernoulliNaiveBayes" + SklearnDummyClassifier = "SklearnDummyClassifier" + SklearnPrediction = "SklearnPrediction" + + class AdvancedSkLearn: + KNNClassifierTrainer = "KNNClassifierTrainer" + KNNRegressorTrainer = "KNNRegressorTrainer" + SVCTrainer = "SVCTrainer" + SVRTrainer = "SVRTrainer" + + class HuggingFace: + HuggingFaceSentimentAnalysis = "HuggingFaceSentimentAnalysis" + HuggingFaceTextSummarization = "HuggingFaceTextSummarization" + HuggingFaceSpamSMSDetection = "HuggingFaceSpamSMSDetection" + HuggingFaceIrisLogisticRegression = "HuggingFaceIrisLogisticRegression" + + class General: + Scorer = "Scorer" + Split = "Split" + SentimentAnalysis = "SentimentAnalysis" + + class UDF: + PythonUDFV2 = "PythonUDFV2" + PythonUDFSourceV2 = "PythonUDFSourceV2" + DualInputPortsPythonUDFV2 = "DualInputPortsPythonUDFV2" + PythonLambdaFunction = "PythonLambdaFunction" + PythonTableReducer = "PythonTableReducer" + + JavaUDF = "JavaUDF" + RUDF = "RUDF" + RUDFSource = "RUDFSource" + + +def get_flat_list(cls): + result = [] + for attr_name in dir(cls): + attr_value = getattr(cls, attr_name) + if not attr_name.startswith("__") and not callable(attr_value): + if isinstance(attr_value, type): + result.extend(get_flat_list(attr_value)) + else: + result.append(attr_value) + return result diff --git a/core/suggestion-service/model/texera/TexeraPort.py b/core/suggestion-service/model/texera/TexeraPort.py new file mode 100644 index 00000000000..4031f12384b --- /dev/null +++ b/core/suggestion-service/model/texera/TexeraPort.py @@ -0,0 +1,83 @@ +from typing import List + +from model.DataSchema import DataSchema +from model.Operator import Operator +from model.Port import Port +from model.llm.interpretation import ( + PortInterpretation, + SchemaInterpretation, + AttributeInterpretation, +) + + +class TexeraPort(Port): + def __init__( + self, + port_dict: dict, + is_input_port: bool, + operator: Operator, + schema: DataSchema = DataSchema([]), + ): + self.port_id = port_dict.get("portID", "") + self.display_name = port_dict.get("displayName", "") + self.allow_multi_inputs = port_dict.get("allowMultiInputs", False) + self.is_dynamic_port = port_dict.get("isDynamicPort", False) + self.dependencies = port_dict.get("dependencies", {}) + self.is_input_port = is_input_port + self.affiliate_operator = operator + self.data_schema = schema + + def IsInputPort(self) -> bool: + return self.is_input_port + + def IsOutputPort(self) -> bool: + return not self.is_input_port + + def GetId(self) -> str: + return self.port_id + + def GetDisplayName(self) -> str: + return self.display_name + + def AllowMultiInputs(self) -> bool: + return self.allow_multi_inputs + + def IsDynamicPort(self) -> bool: + return self.is_dynamic_port + + def GetDependencies(self) -> List[str]: + return self.dependencies + + def GetSourcePorts(self) -> List["Port"]: + if self.IsOutputPort(): + raise RuntimeError("output port doesn't have the source ports!") + return self.source_ports + + def GetTargetPorts(self) -> List["Port"]: + if self.IsInputPort(): + raise RuntimeError("Input port doesn't have the target ports") + return self.target_ports + + def GetDataSchema(self) -> "DataSchema": + return self.data_schema + + def GetAffiliateOperator(self) -> "Operator": + return self.affiliate_operator + + def ToPydantic(self) -> PortInterpretation: + attributeList = [ + AttributeInterpretation(attributeName=attr.name, attributeType=attr.type) + for attr in self.GetDataSchema().attributes + ] + return PortInterpretation( + portID=self.GetId(), + inputSchema=SchemaInterpretation(attributes=attributeList), + ) + + def __str__(self) -> str: + return ( + f"TexeraPort(\n" + f" ID={self.port_id}, \n" + f" DataSchema={self.data_schema}, \n" + f")" + ) diff --git a/core/suggestion-service/model/texera/TexeraWorkflow.py b/core/suggestion-service/model/texera/TexeraWorkflow.py new file mode 100644 index 00000000000..f3e56b61718 --- /dev/null +++ b/core/suggestion-service/model/texera/TexeraWorkflow.py @@ -0,0 +1,426 @@ +import json +from typing import List, Dict, Tuple, Any, Set, Optional +import networkx as nx + +from model.Operator import Operator +from model.Port import Port +from model.DataSchema import DataSchema + +from model.Workflow import Workflow +from model.llm.interpretation import ( + WorkflowInterpretation, + LinkInterpretation, + LinkEndInterpretation, +) +from model.texera.TexeraOperator import TexeraOperator + + +class Link: + """Simple class to represent a link between operators.""" + + def __init__(self, link_id, src_id, src_port, target_id, target_port): + self.link_id = link_id + self.source = type( + "obj", (object,), {"operator_id": src_id, "port_id": src_port} + ) + self.target = type( + "obj", (object,), {"operator_id": target_id, "port_id": target_port} + ) + + +class TexeraWorkflow(Workflow): + def __init__( + self, + workflow_dict: Dict[str, Any] = None, + input_schema: Dict[str, List] = None, + operator_errors: Dict[str, Any] = None, + wid: int = 0, + workflow_title: str = "", + ): + """ + Initialize a TexeraWorkflow from either a workflow dictionary or directly from operators and links. + + Args: + workflow_dict: Dictionary representation of the workflow + input_schema: Dictionary mapping operator IDs to their input schemas + operator_errors: Dictionary mapping operator IDs to their errors + wid: Workflow ID + workflow_title: Title of the workflow + operators: Dictionary of operator IDs to TexeraOperator objects + links: List of link objects + """ + # Initialize with empty values + self.wid = wid + self.workflow_title = workflow_title + self.DAG = nx.DiGraph() + self.operators = {} + self.links = [] + self.input_schema = input_schema or {} + self.operator_errors = operator_errors or {} + self.workflow_dict = None + self.workflow_content = None + + # Initialize based on provided parameters + if workflow_dict is not None: + # Initialize from workflow dictionary + self.workflow_dict = workflow_dict + self.workflow_content = json.dumps(workflow_dict) + self.initialize_from_dict(workflow_dict) + + def initialize_from_dict(self, workflow_dict: Dict[str, Any]) -> None: + """Initialize the workflow from a dictionary.""" + self.workflow_content = json.dumps(workflow_dict) + self.workflow_dict = workflow_dict + + # Extract operators + content = workflow_dict.get("content", {}) + operators_dict = content.get("operators", []) + + # Reset internal state + self.operators = {} + self.links = [] + + # Initialize operators + for op_dict in operators_dict: + op_id = op_dict.get("operatorID") + if op_id: + self.operators[op_id] = TexeraOperator( + operator_dict=op_dict, + port_indexed_input_schemas=self._get_input_schemas_for_operator( + op_id + ), + error=self.operator_errors.get(op_id, {}), + ) + + # Build the DAG (adds links and completes initialization) + self._build_dag() + + def _build_dag(self) -> None: + """Build the DAG from the workflow dictionary's operators and links.""" + self.DAG = nx.DiGraph() + + # Add nodes to DAG + for operator in self.GetOperators(): + self.DAG.add_node( + operator.GetId(), + type=operator.GetType(), + inputPorts=[port.GetId() for port in operator.GetInputPorts()], + outputPorts=[port.GetId() for port in operator.GetOutputPorts()], + error=operator.GetError(), + ) + + # Add links to DAG + links_dict = self.workflow_dict.get("content", {}).get("links", []) + for link in links_dict: + link_id = link.get("linkID") + source_op_id = link.get("source", {}).get("operatorID") + src_port_id = link.get("source", {}).get("portID") + target_op_id = link.get("target", {}).get("operatorID") + target_port_id = link.get("target", {}).get("portID") + + if source_op_id and target_op_id: + # Add link to links list + self.links.append( + Link( + link_id, source_op_id, src_port_id, target_op_id, target_port_id + ) + ) + + # Add edge to DAG + op = self.operators.get(target_op_id) + schema = None + if op is not None: + schema = op.GetInputSchemaByPortID(target_port_id) + + self.DAG.add_edge( + source_op_id, + target_op_id, + srcPort=src_port_id, + targetPort=target_port_id, + schema=schema, + ) + + def _build_dag_from_operators_and_links(self) -> None: + """Build the DAG from the operators and links provided directly.""" + self.DAG = nx.DiGraph() + + # Add nodes to DAG + for operator in self.GetOperators(): + self.DAG.add_node( + operator.GetId(), + type=operator.GetType(), + inputPorts=[port.GetId() for port in operator.GetInputPorts()], + outputPorts=[port.GetId() for port in operator.GetOutputPorts()], + error=operator.GetError(), + ) + + # Add links to DAG + for link in self.links: + source_op_id = link.source.operator_id + src_port_id = link.source.port_id + target_op_id = link.target.operator_id + target_port_id = link.target.port_id + + # Add edge to DAG + op = self.operators.get(target_op_id) + schema = None + if op is not None: + schema = op.GetInputSchemaByPortID(target_port_id) + + self.DAG.add_edge( + source_op_id, + target_op_id, + srcPort=src_port_id, + targetPort=target_port_id, + schema=schema, + ) + + def get_all_paths(self) -> List[List[str]]: + """ + Find all possible paths through the workflow DAG. + A path is a sequence of operator IDs from a source node (no incoming edges) + to a sink node (no outgoing edges). + + Returns: + List of paths, where each path is a list of operator IDs + """ + # Find source and sink nodes + source_nodes = [ + node for node in self.DAG.nodes() if self.DAG.in_degree(node) == 0 + ] + sink_nodes = [ + node for node in self.DAG.nodes() if self.DAG.out_degree(node) == 0 + ] + + if not source_nodes or not sink_nodes: + return [] + + # Use DFS to find all paths + all_paths = [] + + def dfs_paths(current_node, path, visited): + visited.add(current_node) + + # If we've reached a sink node, add the path + if current_node in sink_nodes: + all_paths.append(path.copy()) + + # Explore neighbors + for neighbor in self.DAG.successors(current_node): + if neighbor not in visited: + path.append(neighbor) + dfs_paths(neighbor, path, visited.copy()) + path.pop() + + # Start DFS from each source node + for source in source_nodes: + dfs_paths(source, [source], set()) + + return all_paths + + def extract_path_workflow(self, path: List[str]) -> "TexeraWorkflow": + """Extract a subworkflow containing only the operators and links in the given path.""" + # Create a new workflow dict with only the operators in the path + subworkflow_dict = {"content": {"operators": [], "links": []}} + + # Add operators from the path + for op_id in path: + if op_id in self.operators: + # Find the original operator dict + for op_dict in self.workflow_dict.get("content", {}).get( + "operators", [] + ): + if op_dict.get("operatorID") == op_id: + subworkflow_dict["content"]["operators"].append(op_dict.copy()) + break + + # Add links between operators in the path + for i in range(len(path) - 1): + source_op_id, target_op_id = path[i], path[i + 1] + + # Find links between these operators + for link_dict in self.workflow_dict.get("content", {}).get("links", []): + source = link_dict.get("source", {}) + target = link_dict.get("target", {}) + + if ( + source.get("operatorID") == source_op_id + and target.get("operatorID") == target_op_id + ): + subworkflow_dict["content"]["links"].append(link_dict.copy()) + + # Create a new TexeraWorkflow with the same input schema and errors but only for this path + path_input_schema = { + op_id: self.input_schema.get(op_id, []) + for op_id in path + if op_id in self.input_schema + } + + path_operator_errors = { + op_id: self.operator_errors.get(op_id) + for op_id in path + if op_id in self.operator_errors + } + + return TexeraWorkflow( + workflow_dict=subworkflow_dict, + input_schema=path_input_schema, + operator_errors=path_operator_errors, + wid=self.wid, + workflow_title=f"Path from {path[0]} to {path[-1]}", + ) + + # Interface methods + def GetWorkflowContent(self) -> str: + return self.workflow_content + + def GetWorkflowId(self) -> int: + return self.wid + + def GetOperators(self, types: List[str] = None) -> List["Operator"]: + if types is None: + return list(self.operators.values()) + return [op for op in self.operators.values() if op.GetType() in types] + + def get_operators(self) -> List[Dict[str, Any]]: + """Returns the operator dictionaries from the workflow.""" + return self.workflow_dict.get("content", {}).get("operators", []) + + def get_links(self) -> List[Dict[str, Any]]: + """Returns the link dictionaries from the workflow.""" + return self.workflow_dict.get("content", {}).get("links", []) + + def TopologicalSort(self) -> List["Operator"]: + # This is a placeholder for actual topological sort logic + return list(self.operators.values()) + + def GetDAG(self): + return self.DAG + + def GetSchemaToNextOperatorDistributionMapping( + self, + ) -> Dict["DataSchema", Dict[str, int]]: + result = {} + for source_op_id, target_op_id, edge_data in self.DAG.edges(data=True): + schema = edge_data["schema"] + target_op = self.operators.get(target_op_id) + target_op_type = target_op.GetType() + + if schema not in result: + result[schema] = {} + if target_op_type not in result[schema]: + result[schema][target_op_type] = 0 + result[schema][target_op_type] += 1 + return result + + def GetOperatorTypeToNextOperatorDistributionMapping( + self, + ) -> Dict[str, Dict[str, int]]: + result = {} + for source_op_id, target_op_id, edge_data in self.DAG.edges(data=True): + source_op = self.operators.get(source_op_id) + source_op_type = source_op.GetType() + target_op = self.operators.get(target_op_id) + target_op_type = target_op.GetType() + + if source_op_type not in result: + result[source_op_type] = {} + if target_op_type not in result[source_op_type]: + result[source_op_type][target_op_type] = 0 + result[source_op_type][target_op_type] += 1 + return result + + def GetAdditionPairs( + self, + ) -> List[Tuple[Tuple[Operator, Port], Tuple[Operator, Port]]]: + results = [] + for source_op_id, target_op_id, edge_data in self.DAG.edges(data=True): + source_op = self.operators.get(source_op_id) + target_op = self.operators.get(target_op_id) + srcPortId = edge_data.get("srcPort") + targetPortId = edge_data.get("targetPort") + + if source_op is None or target_op is None: + continue + + srcPort = next( + ( + port + for port in source_op.GetOutputPorts() + if port.GetId() == srcPortId + ), + None, + ) + targetPort = next( + ( + port + for port in target_op.GetInputPorts() + if port.GetId() == targetPortId + ), + None, + ) + + if srcPort is None or targetPort is None: + continue + + results.append(((source_op, srcPort), (target_op, targetPort))) + return results + + def _get_input_schemas_for_operator(self, operator_id: str) -> List["DataSchema"]: + """Extract input schemas for a given operator from the input_schema dictionary.""" + if not self.input_schema or operator_id not in self.input_schema: + return [] + schema_list = self.input_schema.get(operator_id, []) + return [DataSchema(s) for s in schema_list] + + def __str__(self) -> str: + operators_str = "\n".join([str(operator) for operator in self.GetOperators()]) + edges_str = "\n".join( + [f"{source} -> {target}" for source, target in self.DAG.edges()] + ) + return ( + f"TexeraWorkflow(\n" + f" WorkflowID={self.wid},\n" + f" Title={self.workflow_title},\n" + f" Operators=[\n{operators_str}\n ],\n" + f" DAG Edges=[\n {edges_str}\n ]\n" + f")" + ) + + def VisualizeDAG(self) -> str: + """Generate a text-based visualization of the workflow DAG.""" + visualization = "Workflow DAG Visualization:\n\nOperators:\n" + for op_id, operator in self.operators.items(): + visualization += f" - {op_id} ({operator.GetType()})\n" + + visualization += "\nLinks:\n" + for link in self.links: + source_op = link.source.operator_id + source_port = link.source.port_id + target_op = link.target.operator_id + target_port = link.target.port_id + visualization += ( + f" - {source_op}:{source_port} → {target_op}:{target_port}\n" + ) + + return visualization + + def ToPydantic(self) -> WorkflowInterpretation: + return WorkflowInterpretation( + operators={ + op_id: operator.ToPydantic() + for op_id, operator in self.operators.items() + }, + links=[ + LinkInterpretation( + source=LinkEndInterpretation( + operatorID=link.source.operator_id, portID=link.source.port_id + ), + target=LinkEndInterpretation( + operatorID=link.target.operator_id, portID=link.target.port_id + ), + linkID=link.link_id, + ) + for link in self.links + ], + ) diff --git a/core/suggestion-service/model/texera/__init__.py b/core/suggestion-service/model/texera/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/web/__init__.py b/core/suggestion-service/model/web/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/suggestion-service/model/web/input.py b/core/suggestion-service/model/web/input.py new file mode 100644 index 00000000000..1a0e48f8db2 --- /dev/null +++ b/core/suggestion-service/model/web/input.py @@ -0,0 +1,113 @@ +# Input models +import json +import re + +from pydantic import BaseModel, Field, validator, field_validator +from typing import List + +from typing import Dict, List, Any, Optional + +from model.llm.interpretation import AttributeInterpretation + +# Import all relevant classes from the proto definition +from model.proto.edu.uci.ics.amber.engine.architecture.worker import ( + TableProfile, + GlobalProfile, + ColumnProfile, + NumericMatrix, + ColumnIndexList, + GlobalProfileTimes, + ColumnStatistics, +) + + +class SchemaAttribute(BaseModel): + attributeName: str + attributeType: str + + class Config: + extra = "allow" + + +class PhysicalPlan(BaseModel): + # operators IS A LIST ➜ declare it as such + operators: List[Dict[str, Any]] = Field(default_factory=list) + links: List[Dict[str, Any]] = Field(default_factory=list) + + +class CompilationStateInfo(BaseModel): + state: str + physicalPlan: Optional[PhysicalPlan] = None + operatorInputSchemaMap: Optional[ + Dict[str, List[Optional[List[SchemaAttribute]]]] + ] = None + operatorErrors: Optional[Dict[str, Any]] = None + + +class ExecutionStateInfo(BaseModel): + state: str + currentTuples: Optional[Dict[str, Any]] = None + errorMessages: Optional[List[Dict[str, Any]]] = None + + +class SuggestionRequest(BaseModel): + workflow: str = Field(..., description="JSON string of the workflow") + compilationState: CompilationStateInfo + executionState: Optional[ExecutionStateInfo] = None + intention: Optional[str] = Field( + "", description="User intention for the suggestion generation" + ) + focusingOperatorIDs: Optional[List[str]] = Field( + default_factory=list, description="Operator IDs that the user wants to focus on" + ) + operatorIDToTableSchemaMap: Dict[str, List[AttributeInterpretation]] + + +def _camel_to_snake(name: str) -> str: + """ + fooBar → foo_bar + rowStatsMs → row_stats_ms + """ + s1 = re.sub(r"(.)([A-Z][a-z]+)", r"\1_\2", name) + return re.sub(r"([a-z0-9])([A-Z])", r"\1_\2", s1).lower() + + +def _deep_snake(d: Any) -> Any: + """ + Recursively convert all dict keys from camelCase to snake_case. + Lists / scalars are left untouched. + """ + if isinstance(d, dict): + return {_camel_to_snake(k): _deep_snake(v) for k, v in d.items()} + if isinstance(d, list): + return [_deep_snake(x) for x in d] + return d + + +class TableProfileSuggestionRequest(BaseModel): + tableProfile: TableProfile + tableSchema: List[AttributeInterpretation] + targetColumnName: str + focusingOperatorID: str + + # 🔑 custom validator + @field_validator("tableProfile", mode="before") + def _coerce_table_profile(cls, v): + """ + Accept: + • dict (parsed JSON) → snake_case → betterproto + • JSON string → same + • TableProfile → pass through + """ + if isinstance(v, TableProfile): + return v + + if isinstance(v, (dict, str)): + data_dict: Dict[str, Any] = v if isinstance(v, dict) else json.loads(v) + snake = _deep_snake(data_dict) + return TableProfile().from_json(json.dumps(snake)) + + raise TypeError("tableProfile must be a TableProfile, dict, or JSON string.") + + class Config: + arbitrary_types_allowed = True diff --git a/core/suggestion-service/requirements.txt b/core/suggestion-service/requirements.txt new file mode 100644 index 00000000000..90f7ba10874 --- /dev/null +++ b/core/suggestion-service/requirements.txt @@ -0,0 +1,16 @@ +# Core dependencies +networkx==3.1.0 # Used for DAG operations +pyparsing==3.1.1 # Required by pyiceberg + +# API dependencies (for future REST API implementation) +fastapi==0.101.1 +uvicorn==0.23.2 +pydantic==2.1.1 + +# Utility libraries +typing-extensions==4.8.0 # Enhanced typing support +python-dotenv==1.0.0 # For loading environment variables + +# LLM API clients +openai==1.76.0 # OpenAI API client +anthropic==0.50.0 # Anthropic API client \ No newline at end of file diff --git a/core/suggestion-service/suggestion_engine/__init__.py b/core/suggestion-service/suggestion_engine/__init__.py new file mode 100644 index 00000000000..4075d34b667 --- /dev/null +++ b/core/suggestion-service/suggestion_engine/__init__.py @@ -0,0 +1 @@ +# Suggestion engine package diff --git a/core/suggestion-service/suggestion_engine/generator.py b/core/suggestion-service/suggestion_engine/generator.py new file mode 100644 index 00000000000..6c7f5e79f6e --- /dev/null +++ b/core/suggestion-service/suggestion_engine/generator.py @@ -0,0 +1,297 @@ +from typing import Dict, List, Any, Optional +import json +import uuid +import os +from dotenv import load_dotenv + +from model.llm.suggestion import SuggestionList, DataCleaningSuggestionList +from model.web.input import ( + CompilationStateInfo, + ExecutionStateInfo, + TableProfileSuggestionRequest, + SuggestionRequest, +) +from workflow_interpretation.interpreter import ( + WorkflowInterpreter, +) +from model.llm.prompt import SuggestionPrompt, DataCleaningSuggestionPrompt +from model.llm.interpretation import ( + PathInterpretation, + RawInterpretation, + OperatorInterpretation, + BaseInterpretation, + InterpretationMethod, + AttributeInterpretation, + SchemaInterpretation, +) +from llm_agent.base import LLMAgentFactory +from distutils.util import strtobool # stdlib helper + +# Load environment variables from .env file if present +load_dotenv() + +logs_directory = "logs" + + +# helper that treats 1/true/yes/y (case‑insensitive) as True +def env_bool(key: str, default: bool = False) -> bool: + try: + return bool(strtobool(os.getenv(key, str(default)))) + except ValueError: + return default + + +class SuggestionGenerator: + """ + SuggestionGenerator is responsible for generating workflow suggestions + based on the current workflow state, compilation information, and result data. + """ + + def __init__(self): + """ + Initialize the suggestion generator. + """ + self.workflow_interpretation_method = InterpretationMethod( + os.environ.get("INTERPRETATION_METHOD") + ) + self.workflow_interpreter = WorkflowInterpreter( + self.workflow_interpretation_method + ) + + # Determine provider and model + self.llm_provider = os.environ.get("LLM_PROVIDER") + + if self.llm_provider == "openai": + self.llm_model = os.environ.get("OPENAI_MODEL") + elif self.llm_provider == "anthropic": + self.llm_model = os.environ.get("ANTHROPIC_MODEL") + + # Create the LLM agent + try: + extra_params = {} + + if self.llm_provider == "openai": + tools = [] + vector_store_ids_raw = os.environ.get("OPENAI_VECTOR_STORE_IDS", "") + vector_store_ids = [ + v.strip() for v in vector_store_ids_raw.split(",") if v.strip() + ] + if vector_store_ids: + tools.append( + {"type": "file_search", "vector_store_ids": vector_store_ids} + ) + extra_params["tools"] = tools + extra_params["project"] = os.environ.get("OPENAI_PROJECT_ID") + extra_params["organization"] = os.environ.get("OPENAI_ORG_ID") + extra_params["use_function_calls"] = env_bool( + "OPENAI_USE_FUNCTION_CALLS", default=True + ) + self.llm_agent = LLMAgentFactory.create( + self.llm_provider, model=self.llm_model, **extra_params + ) + except ValueError as e: + print(f"Error creating LLM agent: {str(e)}") + self.llm_agent = None + + def generate_suggestions( + self, + request: SuggestionRequest, + enable_logging: bool = True, + ) -> SuggestionList: + """ + Generate workflow suggestions based on the current workflow, compilation state, execution state, and result tables. + + Returns: + A list of workflow suggestions + """ + # If LLM generation failed or agent is not available, return mock suggestions + if not self.llm_agent: + return SuggestionList(suggestions=[]) + workflow_json = json.loads(request.workflow) + # Generate natural language description of the workflow + interpretation = self.workflow_interpreter.interpret_workflow( + workflow_json, + request.compilationState, + ) + + # Determine intention (fallback) + intention = request.intention + if not intention: + intention = "Recommend improvements and fixes of current workflows" + + workflow_intp = interpretation.get_base_workflow_interpretation() + focusing_operators: List[OperatorInterpretation] = [] + for oid in request.focusingOperatorIDs: + op = workflow_intp.operators.get(oid) + if op: + output_attributes = request.operatorIDToTableSchemaMap.get(oid, []) + op.outputSchema = SchemaInterpretation(attributes=output_attributes) + focusing_operators.append(op) + + prompt_obj = SuggestionPrompt( + intention=intention, + focusingOperators=focusing_operators, + workflowInterpretation=interpretation, + ) + + # Serialize prompt to JSON string to feed into LLM + workflow_description = prompt_obj.model_dump_json(indent=2) + + log_id: Optional[str] = None + # Save the workflow description to a log file if logging is enabled + if enable_logging: + try: + os.makedirs(logs_directory, exist_ok=True) + log_id = str(uuid.uuid4()) + file_name = f"{log_id}.json" + file_path = os.path.join(logs_directory, file_name) + with open(file_path, "w", encoding="utf-8") as f: + f.write(workflow_description) + except Exception as e: + print(f"Failed to log workflow description: {e}") + + # Get suggestions from the LLM agent + suggestions = self.llm_agent.generate_suggestions( + prompt=prompt_obj, + temperature=0.7, # Lower temperature for more focused suggestions + ) + if enable_logging and log_id: + try: + with open( + os.path.join(logs_directory, f"response-{log_id}.json"), + "w", + encoding="utf-8", + ) as f: + f.write(suggestions.model_dump_json(indent=2)) + except Exception as e: + print(f"Failed to log suggestions: {e}") + return suggestions + + def generate_data_cleaning_suggestions( + self, + request: TableProfileSuggestionRequest, + enable_logging: bool = True, + ) -> DataCleaningSuggestionList: + """ + Build a DataCleaningSuggestionPrompt from the table profile + target column + and ask the LLM agent for cleaning recommendations. + + Args: + request: TableProfileSuggestionRequest containing full table profile + and the column we need to clean. + enable_logging: if True, raw prompt / response JSON are written to + the `logs/` folder for offline inspection. + + Returns: + DataCleaningSuggestionList with zero or more natural-language + cleaning suggestions. + """ + # Bail out early if no LLM agent has been configured. + if not self.llm_agent: + return DataCleaningSuggestionList(suggestions=[]) + + tp = request.tableProfile # proto TableProfile + target_col = request.targetColumnName + + # -------------------- 1️⃣ locate the target ColumnProfile ------------ + target_profile = None + for col in tp.column_profiles: + # ColumnProfile in proto keeps the original field name `columnName` + if getattr(col, "column_name", None) == target_col: + target_profile = col + break + + # If not found we cannot proceed – return empty result. + if target_profile is None: + return DataCleaningSuggestionList(suggestions=[]) + + # -------------------- 3️⃣ compose the cleaning prompt ---------------- + prompt_obj = DataCleaningSuggestionPrompt( + focusingOperatorID=request.focusingOperatorID, # not available in this request + columnProfile=target_profile, + tableSchema=SchemaInterpretation(attributes=request.tableSchema), + ) + + prompt_json = prompt_obj.model_dump_json(indent=2) + + # Optional logging for debugging + log_id: Optional[str] = None + if enable_logging: + try: + os.makedirs(logs_directory, exist_ok=True) + log_id = str(uuid.uuid4()) + with open( + os.path.join(logs_directory, f"cleaning-{log_id}.json"), + "w", + encoding="utf-8", + ) as f: + f.write(prompt_json) + except Exception as e: + print(f"Failed to log cleaning prompt: {e}") + + # -------------------- 4️⃣ call the LLM agent ------------------------- + suggestions = self.llm_agent.generate_data_cleaning_suggestions( + prompt=prompt_obj, + temperature=0.4, # a bit more deterministic + ) + + if enable_logging and log_id: + try: + with open( + os.path.join(logs_directory, f"cleaning-response-{log_id}.json"), + "w", + encoding="utf-8", + ) as f: + f.write(suggestions.model_dump_json(indent=2)) + except Exception as e: + print(f"Failed to log cleaning suggestions: {e}") + + return suggestions + + def _enhance_prompt_with_state_info( + self, + workflow_description: str, + compilation_state: Dict[str, Any], + execution_state: Optional[Dict[str, Any]], + ) -> str: + """ + Enhance the workflow description with compilation and execution state information. + + Args: + workflow_description: Natural language description of the workflow + compilation_state: Compilation information and errors + execution_state: Current execution state of the workflow + + Returns: + Enhanced workflow description + """ + prompt = workflow_description + "\n\n" + + # Add compilation state info + prompt += f"Compilation State: {compilation_state['state']}\n" + + # Add compilation errors if any + if compilation_state["state"] == "Failed" and compilation_state.get( + "operatorErrors" + ): + prompt += "Compilation Errors:\n" + for op_id, error in compilation_state["operatorErrors"].items(): + if error: + prompt += f"- Operator {op_id}: {error}\n" + + # Add execution state info if available + if execution_state: + prompt += f"\nExecution State: {execution_state['state']}\n" + + # Add execution errors if any + if execution_state["state"] == "Failed" and execution_state.get( + "errorMessages" + ): + prompt += "Execution Errors:\n" + for error in execution_state["errorMessages"]: + prompt += f"- {error}\n" + + # Add final instruction + prompt += "\nBased on this workflow description and state information, suggest improvements or fixes." + + return prompt diff --git a/core/suggestion-service/suggestion_service_test.py b/core/suggestion-service/suggestion_service_test.py new file mode 100644 index 00000000000..b514ce2b757 --- /dev/null +++ b/core/suggestion-service/suggestion_service_test.py @@ -0,0 +1,161 @@ +""" +Workflow Interpretation and Suggestion CLI Utility +""" + +import json +import os +from typing import List, Optional +from pathlib import Path + +from dotenv import load_dotenv + +from workflow_interpretation.interpreter import ( + WorkflowInterpreter, +) +from model.llm.interpretation import InterpretationMethod +from suggestion_engine.generator import SuggestionGenerator +from model.web.input import SuggestionRequest, CompilationStateInfo, ExecutionStateInfo + +# Base directory for test data and results +DATA_DIR = Path("test/data") +RESULTS_DIR = Path("test/results") +load_dotenv() + + +def load_workflow_data(dir_name: str) -> SuggestionRequest: + """ + Load workflow-related JSON files from a given directory. + + Args: + dir_name: Subdirectory name under test/data + + Returns: + SuggestionRequest object containing all loaded workflow components. + """ + path = DATA_DIR / dir_name + + workflow = open(path / "workflow.json").read() + compilation_state = json.load(open(path / "workflow_compilation_state.json")) + execution_state = json.load(open(path / "execution_state.json")) + + return SuggestionRequest( + workflow=workflow, + compilationState=CompilationStateInfo.model_validate(compilation_state), + executionState=ExecutionStateInfo.model_validate(execution_state), + ) + + +def interpret_workflow(dir_name: str): + """ + Run interpretation on a workflow and save the output. + + Args: + dir_name: Workflow folder name + """ + print(f"\n🧠 Interpreting workflow: {dir_name}") + + request = load_workflow_data(dir_name) + + result_dir = RESULTS_DIR / dir_name + os.makedirs(result_dir, exist_ok=True) + + # RAW + raw_text = WorkflowInterpreter(InterpretationMethod.RAW).interpret_workflow( + json.loads(request.workflow), request.compilationState + ) + with open(result_dir / "raw_interpretation.txt", "w") as f: + f.write(raw_text.model_dump_json()) + + print("\n📄 Raw interpretation saved.") + + # BY_PATH + by_path_text = WorkflowInterpreter(InterpretationMethod.BY_PATH).interpret_workflow( + json.loads(request.workflow), request.compilationState + ) + with open(result_dir / "by_path_interpretation.txt", "w") as f: + f.write(by_path_text.model_dump_json()) + + print("📄 By-path interpretation saved.") + + +def generate_suggestions( + dir_name: str, + intention: str = "", + focusing_operator_ids: Optional[List[str]] = None, +): + """ + Generate suggestions from a workflow. + + Args: + dir_name: Workflow folder name + intention: User's intention for the suggestion + focusing_operator_ids: List of operator IDs to focus on + """ + print(f"\n💡 Generating suggestions for: {dir_name}") + print(f" Intention: {intention or '(empty)'}") + print(f" Focusing operators: {focusing_operator_ids or '[]'}") + + request = load_workflow_data(dir_name) + generator = SuggestionGenerator() + + result_dir = RESULTS_DIR / dir_name + os.makedirs(result_dir, exist_ok=True) + + # Update request with any custom parameters + if intention: + request.intention = intention + if focusing_operator_ids: + request.focusingOperatorIDs = focusing_operator_ids + + suggestions = generator.generate_suggestions( + request.workflow, + request.compilationState, + request.executionState, + request.intention, + request.focusingOperatorIDs, + ) + + output_file = result_dir / "suggestions.json" + with open(output_file, "w") as f: + f.write(suggestions.model_dump_json(indent=2)) + + print(f"✅ {len(suggestions.suggestions)} suggestions written to {output_file}") + + +def run_all(): + """Run interpretation and suggestion on all workflows with specific test cases.""" + # Dictionary of test cases with their specific parameters + test_cases = { + "workflow1": { + "intention": "", + "focusing_operator_ids": [], + }, + "workflow2": { + "intention": "Suggest good recommendation techniques", + "focusing_operator_ids": [ + "PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289" + ], + }, + "workflow3": { + "intention": "Set the parameter of the operator to do visualization correctly", + "focusing_operator_ids": [ + "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865" + ], + }, + } + + os.makedirs(RESULTS_DIR, exist_ok=True) + + for dir_name, params in test_cases.items(): + print(f"\n{'='*60}") + print(f"▶ Running for workflow: {dir_name}") + print(f"{'='*60}") + + interpret_workflow(dir_name) + generate_suggestions( + dir_name, params["intention"], params["focusing_operator_ids"] + ) + + +if __name__ == "__main__": + run_all() diff --git a/core/suggestion-service/test/data/workflow1/execution_state.json b/core/suggestion-service/test/data/workflow1/execution_state.json new file mode 100644 index 00000000000..6f885c3d09f --- /dev/null +++ b/core/suggestion-service/test/data/workflow1/execution_state.json @@ -0,0 +1,5 @@ +{ + "state": "Uninitialized", + "currentTuples": null, + "errorMessages": null +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow1/result_tables.json b/core/suggestion-service/test/data/workflow1/result_tables.json new file mode 100644 index 00000000000..9e26dfeeb6e --- /dev/null +++ b/core/suggestion-service/test/data/workflow1/result_tables.json @@ -0,0 +1 @@ +{} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow1/workflow.json b/core/suggestion-service/test/data/workflow1/workflow.json new file mode 100644 index 00000000000..1e5bb4fe2c5 --- /dev/null +++ b/core/suggestion-service/test/data/workflow1/workflow.json @@ -0,0 +1,162 @@ +{ + "wid": 2748, + "name": "Suggestion Test: 1-path-compilation-succeed", + "description": null, + "content": { + "operators": [ + { + "operatorID": "CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e", + "operatorType": "CSVFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "fileEncoding": "UTF_8", + "customDelimiter": ",", + "hasHeader": true, + "fileName": "/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv" + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "CSV File Scan", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "alias": "", + "originalAttribute": "tweet_id" + }, + { + "alias": "", + "originalAttribute": "create_at_month" + }, + { + "alias": "", + "originalAttribute": "favorite_count" + }, + { + "alias": "", + "originalAttribute": "retweet_count" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3", + "operatorType": "Sort", + "operatorVersion": "N/A", + "operatorProperties": { + "attributes": [ + { + "attribute": "favorite_count", + "sortPreference": "DESC" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sort", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + } + ], + "operatorPositions": { + "CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e": { + "x": 472, + "y": 396 + }, + "Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894": { + "x": 657, + "y": 395 + }, + "Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3": { + "x": 825, + "y": 396 + } + }, + "links": [ + { + "linkID": "link-4a444193-d4a7-4a2d-ac53-3425bc5bfda0", + "source": { + "operatorID": "CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e", + "portID": "output-0" + }, + "target": { + "operatorID": "Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894", + "portID": "input-0" + } + }, + { + "linkID": "link-1586b353-1954-4064-a38a-3c9539306c07", + "source": { + "operatorID": "Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894", + "portID": "output-0" + }, + "target": { + "operatorID": "Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3", + "portID": "input-0" + } + } + ], + "commentBoxes": [], + "settings": { + "dataTransferBatchSize": 400 + } + }, + "creationTime": 1744844136205, + "lastModifiedTime": 1744844136205, + "isPublic": false +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow1/workflow_compilation_state.json b/core/suggestion-service/test/data/workflow1/workflow_compilation_state.json new file mode 100644 index 00000000000..3356e0f70b0 --- /dev/null +++ b/core/suggestion-service/test/data/workflow1/workflow_compilation_state.json @@ -0,0 +1,64 @@ +{ + "state": "Succeeded", + "physicalPlan": null, + "operatorInputSchemaMap": { + "Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + }, + { + "attributeName": "is_retweet", + "attributeType": "boolean" + }, + { + "attributeName": "lang", + "attributeType": "string" + }, + { + "attributeName": "text", + "attributeType": "string" + }, + { + "attributeName": "user_id", + "attributeType": "integer" + } + ] + ], + "Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + } + ] + ], + "CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e": [] + }, + "operatorErrors": {} +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow2/execution_state.json b/core/suggestion-service/test/data/workflow2/execution_state.json new file mode 100644 index 00000000000..6f885c3d09f --- /dev/null +++ b/core/suggestion-service/test/data/workflow2/execution_state.json @@ -0,0 +1,5 @@ +{ + "state": "Uninitialized", + "currentTuples": null, + "errorMessages": null +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow2/result_tables.json b/core/suggestion-service/test/data/workflow2/result_tables.json new file mode 100644 index 00000000000..9e26dfeeb6e --- /dev/null +++ b/core/suggestion-service/test/data/workflow2/result_tables.json @@ -0,0 +1 @@ +{} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow2/workflow.json b/core/suggestion-service/test/data/workflow2/workflow.json new file mode 100644 index 00000000000..9ae537b5d5b --- /dev/null +++ b/core/suggestion-service/test/data/workflow2/workflow.json @@ -0,0 +1,267 @@ +{ + "name": "Suggestion Test: 2-path-compilation-succeed", + "description": null, + "wid": 2749, + "content": { + "operators": [ + { + "operatorID": "CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef", + "operatorType": "CSVFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "fileEncoding": "UTF_8", + "customDelimiter": ",", + "hasHeader": true, + "fileName": "/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv" + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "CSV File Scan", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "alias": "", + "originalAttribute": "tweet_id" + }, + { + "alias": "", + "originalAttribute": "create_at_month" + }, + { + "alias": "", + "originalAttribute": "favorite_count" + }, + { + "alias": "", + "originalAttribute": "retweet_count" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260", + "operatorType": "Sort", + "operatorVersion": "N/A", + "operatorProperties": { + "attributes": [ + { + "attribute": "favorite_count", + "sortPreference": "DESC" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sort", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "alias": "", + "originalAttribute": "text" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289", + "operatorType": "PythonUDFV2", + "operatorVersion": "N/A", + "operatorProperties": { + "code": "# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "workers": 1, + "retainInputColumns": true, + "outputColumns": [ + { + "attributeName": "text_length", + "attributeType": "integer" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Python UDF", + "dynamicInputPorts": true, + "dynamicOutputPorts": true + } + ], + "operatorPositions": { + "CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef": { + "x": 472, + "y": 396 + }, + "Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f": { + "x": 657, + "y": 363 + }, + "Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260": { + "x": 816, + "y": 362 + }, + "Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac": { + "x": 661, + "y": 501 + }, + "PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289": { + "x": 820, + "y": 497 + } + }, + "links": [ + { + "linkID": "link-4a444193-d4a7-4a2d-ac53-3425bc5bfda0", + "source": { + "operatorID": "CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef", + "portID": "output-0" + }, + "target": { + "operatorID": "Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f", + "portID": "input-0" + } + }, + { + "linkID": "link-1586b353-1954-4064-a38a-3c9539306c07", + "source": { + "operatorID": "Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f", + "portID": "output-0" + }, + "target": { + "operatorID": "Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260", + "portID": "input-0" + } + }, + { + "linkID": "63989c89-c9e7-470c-b107-bc6d154b2ed0", + "source": { + "operatorID": "CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef", + "portID": "output-0" + }, + "target": { + "operatorID": "Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac", + "portID": "input-0" + } + }, + { + "linkID": "710b40d5-3142-4970-ae65-3ee210690f6f", + "source": { + "operatorID": "Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac", + "portID": "output-0" + }, + "target": { + "operatorID": "PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289", + "portID": "input-0" + } + } + ], + "commentBoxes": [], + "settings": { + "dataTransferBatchSize": 400 + } + }, + "creationTime": 1744845032285, + "lastModifiedTime": 1744845032285, + "isPublished": false, + "readonly": false +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow2/workflow_compilation_state.json b/core/suggestion-service/test/data/workflow2/workflow_compilation_state.json new file mode 100644 index 00000000000..6a42f30864d --- /dev/null +++ b/core/suggestion-service/test/data/workflow2/workflow_compilation_state.json @@ -0,0 +1,108 @@ +{ + "state": "Succeeded", + "physicalPlan": null, + "operatorInputSchemaMap": { + "CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef": [], + "Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + }, + { + "attributeName": "is_retweet", + "attributeType": "boolean" + }, + { + "attributeName": "lang", + "attributeType": "string" + }, + { + "attributeName": "text", + "attributeType": "string" + }, + { + "attributeName": "user_id", + "attributeType": "integer" + } + ] + ], + "PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289": [ + [ + { + "attributeName": "text", + "attributeType": "string" + } + ] + ], + "Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + } + ] + ], + "Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + }, + { + "attributeName": "is_retweet", + "attributeType": "boolean" + }, + { + "attributeName": "lang", + "attributeType": "string" + }, + { + "attributeName": "text", + "attributeType": "string" + }, + { + "attributeName": "user_id", + "attributeType": "integer" + } + ] + ] + }, + "operatorErrors": {} +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow3/execution_state.json b/core/suggestion-service/test/data/workflow3/execution_state.json new file mode 100644 index 00000000000..6f885c3d09f --- /dev/null +++ b/core/suggestion-service/test/data/workflow3/execution_state.json @@ -0,0 +1,5 @@ +{ + "state": "Uninitialized", + "currentTuples": null, + "errorMessages": null +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow3/result_tables.json b/core/suggestion-service/test/data/workflow3/result_tables.json new file mode 100644 index 00000000000..9e26dfeeb6e --- /dev/null +++ b/core/suggestion-service/test/data/workflow3/result_tables.json @@ -0,0 +1 @@ +{} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow3/workflow.json b/core/suggestion-service/test/data/workflow3/workflow.json new file mode 100644 index 00000000000..fcbfa52184e --- /dev/null +++ b/core/suggestion-service/test/data/workflow3/workflow.json @@ -0,0 +1,313 @@ +{ + "name": "Suggestion Test: 2-path-compilation-failed", + "description": null, + "wid": 2750, + "content": { + "operators": [ + { + "operatorID": "CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1", + "operatorType": "CSVFileScan", + "operatorVersion": "N/A", + "operatorProperties": { + "fileEncoding": "UTF_8", + "customDelimiter": ",", + "hasHeader": true, + "fileName": "/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv" + }, + "inputPorts": [], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "CSV File Scan", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "alias": "", + "originalAttribute": "tweet_id" + }, + { + "alias": "", + "originalAttribute": "create_at_month" + }, + { + "alias": "", + "originalAttribute": "favorite_count" + }, + { + "alias": "", + "originalAttribute": "retweet_count" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2", + "operatorType": "Sort", + "operatorVersion": "N/A", + "operatorProperties": { + "attributes": [ + { + "attribute": "favorite_count", + "sortPreference": "DESC" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Sort", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16", + "operatorType": "Projection", + "operatorVersion": "N/A", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "alias": "", + "originalAttribute": "text" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Projection", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + }, + { + "operatorID": "PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c", + "operatorType": "PythonUDFV2", + "operatorVersion": "N/A", + "operatorProperties": { + "code": "# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n", + "workers": 1, + "retainInputColumns": true, + "outputColumns": [ + { + "attributeName": "text_length", + "attributeType": "integer" + } + ] + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": true, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Python UDF", + "dynamicInputPorts": true, + "dynamicOutputPorts": true + }, + { + "operatorID": "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865", + "operatorType": "LineChart", + "operatorVersion": "N/A", + "operatorProperties": { + "yLabel": "Y Axis", + "xLabel": "X Axis" + }, + "inputPorts": [ + { + "portID": "input-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false, + "dependencies": [] + } + ], + "outputPorts": [ + { + "portID": "output-0", + "displayName": "", + "allowMultiInputs": false, + "isDynamicPort": false + } + ], + "showAdvanced": false, + "isDisabled": false, + "customDisplayName": "Line Chart", + "dynamicInputPorts": false, + "dynamicOutputPorts": false + } + ], + "operatorPositions": { + "CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1": { + "x": 472, + "y": 396 + }, + "Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c": { + "x": 657, + "y": 363 + }, + "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2": { + "x": 816, + "y": 362 + }, + "Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16": { + "x": 661, + "y": 501 + }, + "PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c": { + "x": 820, + "y": 497 + }, + "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865": { + "x": 966, + "y": 357 + } + }, + "links": [ + { + "linkID": "link-4a444193-d4a7-4a2d-ac53-3425bc5bfda0", + "source": { + "operatorID": "CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1", + "portID": "output-0" + }, + "target": { + "operatorID": "Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c", + "portID": "input-0" + } + }, + { + "linkID": "link-1586b353-1954-4064-a38a-3c9539306c07", + "source": { + "operatorID": "Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c", + "portID": "output-0" + }, + "target": { + "operatorID": "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2", + "portID": "input-0" + } + }, + { + "linkID": "63989c89-c9e7-470c-b107-bc6d154b2ed0", + "source": { + "operatorID": "CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1", + "portID": "output-0" + }, + "target": { + "operatorID": "Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16", + "portID": "input-0" + } + }, + { + "linkID": "710b40d5-3142-4970-ae65-3ee210690f6f", + "source": { + "operatorID": "Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16", + "portID": "output-0" + }, + "target": { + "operatorID": "PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c", + "portID": "input-0" + } + }, + { + "linkID": "link-2883d565-72d0-4d01-a210-fead4ad076a3", + "source": { + "operatorID": "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2", + "portID": "output-0" + }, + "target": { + "operatorID": "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865", + "portID": "input-0" + } + } + ], + "commentBoxes": [], + "settings": { + "dataTransferBatchSize": 400 + } + }, + "creationTime": 1744845299552, + "lastModifiedTime": 1744845299552, + "isPublished": false, + "readonly": false +} \ No newline at end of file diff --git a/core/suggestion-service/test/data/workflow3/workflow_compilation_state.json b/core/suggestion-service/test/data/workflow3/workflow_compilation_state.json new file mode 100644 index 00000000000..d849531024e --- /dev/null +++ b/core/suggestion-service/test/data/workflow3/workflow_compilation_state.json @@ -0,0 +1,150 @@ +{ + "state": "Failed", + "physicalPlan": null, + "operatorInputSchemaMap": { + "Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + }, + { + "attributeName": "is_retweet", + "attributeType": "boolean" + }, + { + "attributeName": "lang", + "attributeType": "string" + }, + { + "attributeName": "text", + "attributeType": "string" + }, + { + "attributeName": "user_id", + "attributeType": "integer" + } + ] + ], + "CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1": [], + "PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c": [ + [ + { + "attributeName": "text", + "attributeType": "string" + } + ] + ], + "Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + } + ] + ], + "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + } + ] + ], + "Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16": [ + [ + { + "attributeName": "tweet_id", + "attributeType": "integer" + }, + { + "attributeName": "create_at_month", + "attributeType": "timestamp" + }, + { + "attributeName": "favorite_count", + "attributeType": "integer" + }, + { + "attributeName": "retweet_count", + "attributeType": "integer" + }, + { + "attributeName": "is_retweet", + "attributeType": "boolean" + }, + { + "attributeName": "lang", + "attributeType": "string" + }, + { + "attributeName": "text", + "attributeType": "string" + }, + { + "attributeName": "user_id", + "attributeType": "integer" + } + ] + ] + }, + "operatorErrors": { + "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865": { + "type": { + "value": 0, + "index": 0, + "name": "COMPILATION_ERROR", + "compilationError": true, + "executionFailure": false, + "unrecognized": false + }, + "timestamp": { + "seconds": 1744853005, + "nanos": 636787000, + "unknownFields": { + "fields": {} + } + }, + "message": "java.lang.RuntimeException: Operator is not configured properly: null", + "details": "Stack trace for developers: \n\njava.lang.RuntimeException: Operator is not configured properly: null\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$9(WorkflowCompiler.scala:149)\nscala.Option.foreach(Option.scala:437)\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$3(WorkflowCompiler.scala:146)\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$3$adapted(WorkflowCompiler.scala:119)\nscala.collection.IterableOnceOps.foreach(IterableOnce.scala:576)\nscala.collection.IterableOnceOps.foreach$(IterableOnce.scala:574)\nscala.collection.AbstractIterator.foreach(Iterator.scala:1300)\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$1(WorkflowCompiler.scala:119)\nscala.collection.IterableOnceOps.foreach(IterableOnce.scala:576)\nscala.collection.IterableOnceOps.foreach$(IterableOnce.scala:574)\nscala.collection.AbstractIterator.foreach(Iterator.scala:1300)\nedu.uci.ics.amber.compiler.WorkflowCompiler.expandLogicalPlan(WorkflowCompiler.scala:109)\nedu.uci.ics.amber.compiler.WorkflowCompiler.compile(WorkflowCompiler.scala:188)\nedu.uci.ics.texera.service.resource.WorkflowCompilationResource.compileWorkflow(WorkflowCompilationResource.scala:52)\njdk.internal.reflect.GeneratedMethodAccessor44.invoke(Unknown Source)\njava.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.base/java.lang.reflect.Method.invoke(Method.java:566)\norg.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory.lambda$static$0(ResourceMethodInvocationHandlerFactory.java:52)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher$1.run(AbstractJavaResourceMethodDispatcher.java:146)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.invoke(AbstractJavaResourceMethodDispatcher.java:189)\norg.glassfish.jersey.server.model.internal.JavaResourceMethodDispatcherProvider$TypeOutInvoker.doDispatch(JavaResourceMethodDispatcherProvider.java:219)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.dispatch(AbstractJavaResourceMethodDispatcher.java:93)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.invoke(ResourceMethodInvoker.java:478)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:400)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:81)\norg.glassfish.jersey.server.ServerRuntime$1.run(ServerRuntime.java:256)\norg.glassfish.jersey.internal.Errors$1.call(Errors.java:248)\norg.glassfish.jersey.internal.Errors$1.call(Errors.java:244)\norg.glassfish.jersey.internal.Errors.process(Errors.java:292)\norg.glassfish.jersey.internal.Errors.process(Errors.java:274)\norg.glassfish.jersey.internal.Errors.process(Errors.java:244)\norg.glassfish.jersey.process.internal.RequestScope.runInScope(RequestScope.java:265)\norg.glassfish.jersey.server.ServerRuntime.process(ServerRuntime.java:235)\norg.glassfish.jersey.server.ApplicationHandler.handle(ApplicationHandler.java:684)\norg.glassfish.jersey.servlet.WebComponent.serviceImpl(WebComponent.java:394)\norg.glassfish.jersey.servlet.WebComponent.service(WebComponent.java:346)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:358)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:311)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:205)\norg.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:764)\norg.eclipse.jetty.servlet.ServletHandler$ChainEnd.doFilter(ServletHandler.java:1665)\nio.dropwizard.servlets.ThreadNameFilter.doFilter(ThreadNameFilter.java:36)\norg.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:202)\norg.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1635)\nio.dropwizard.jersey.filter.AllowedMethodsFilter.handle(AllowedMethodsFilter.java:46)\nio.dropwizard.jersey.filter.AllowedMethodsFilter.doFilter(AllowedMethodsFilter.java:40)\norg.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:202)\norg.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1635)\norg.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:527)\norg.eclipse.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:221)\norg.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1381)\norg.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:176)\norg.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:484)\norg.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:174)\norg.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1303)\norg.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:129)\norg.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:122)\nio.dropwizard.metrics.jetty11.InstrumentedHandler.handle(InstrumentedHandler.java:313)\nio.dropwizard.jetty.RoutingHandler.handle(RoutingHandler.java:52)\norg.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:122)\norg.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:822)\nio.dropwizard.jetty.ZipExceptionHandlingGzipHandler.handle(ZipExceptionHandlingGzipHandler.java:26)\norg.eclipse.jetty.server.handler.RequestLogHandler.handle(RequestLogHandler.java:46)\norg.eclipse.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:173)\norg.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:122)\norg.eclipse.jetty.server.Server.handle(Server.java:563)\norg.eclipse.jetty.server.HttpChannel$RequestDispatchable.dispatch(HttpChannel.java:1598)\norg.eclipse.jetty.server.HttpChannel.dispatch(HttpChannel.java:753)\norg.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:501)\norg.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:287)\norg.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:314)\norg.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:100)\norg.eclipse.jetty.io.SelectableChannelEndPoint$1.run(SelectableChannelEndPoint.java:53)\norg.eclipse.jetty.util.thread.strategy.AdaptiveExecutionStrategy.runTask(AdaptiveExecutionStrategy.java:421)\norg.eclipse.jetty.util.thread.strategy.AdaptiveExecutionStrategy.consumeTask(AdaptiveExecutionStrategy.java:390)\norg.eclipse.jetty.util.thread.strategy.AdaptiveExecutionStrategy.tryProduce(AdaptiveExecutionStrategy.java:277)\norg.eclipse.jetty.util.thread.strategy.AdaptiveExecutionStrategy.run(AdaptiveExecutionStrategy.java:199)\norg.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:411)\norg.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:969)\norg.eclipse.jetty.util.thread.QueuedThreadPool$Runner.doRunJob(QueuedThreadPool.java:1194)\norg.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1149)\njava.base/java.lang.Thread.run(Thread.java:829)", + "operatorId": "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865", + "workerId": "" + } + } +} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow1/by_path_interpretation.txt b/core/suggestion-service/test/results/workflow1/by_path_interpretation.txt new file mode 100644 index 00000000000..586fbbd6a28 --- /dev/null +++ b/core/suggestion-service/test/results/workflow1/by_path_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e":{"operatorID":"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3":{"operatorID":"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e","portID":"output-0"},"target":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","portID":"output-0"},"target":{"operatorID":"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3","portID":"input-0"}}]},"paths":[["CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e","Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3"]]} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow1/raw_interpretation.txt b/core/suggestion-service/test/results/workflow1/raw_interpretation.txt new file mode 100644 index 00000000000..c58135e9733 --- /dev/null +++ b/core/suggestion-service/test/results/workflow1/raw_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e":{"operatorID":"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3":{"operatorID":"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-02af511d-2912-4d48-a86e-8ac73a6e2b0e","portID":"output-0"},"target":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-9d955369-5e4d-44f9-a6f3-ec66b1192894","portID":"output-0"},"target":{"operatorID":"Sort-operator-860e9211-556a-4789-ab06-fa79bc0b36c3","portID":"input-0"}}]}} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow1/suggestions.json b/core/suggestion-service/test/results/workflow1/suggestions.json new file mode 100644 index 00000000000..9322cf967a0 --- /dev/null +++ b/core/suggestion-service/test/results/workflow1/suggestions.json @@ -0,0 +1,125 @@ +{ + "suggestions": [ + { + "suggestion": "Add a FileScan operator to enable reading input data from a specified file.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "FileScan", + "operatorID": "fileScan1", + "operatorProperties": { + "fileName": "your_file.csv", + "encoding": "UTF_8", + "extract": false, + "outputFileName": false, + "attributeType": "string", + "attributeName": "line" + }, + "customDisplayName": null + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + }, + { + "suggestion": "Add a Sort operator to allow sorting of data on specified attributes.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "Sort", + "operatorID": "sort1", + "operatorProperties": { + "attributes": [ + { + "attribute": "your_attribute", + "sortPreference": "ASC" + } + ] + }, + "customDisplayName": null + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + }, + { + "suggestion": "Introduce a Projection operator to select or drop specific columns from the data.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "Projection", + "operatorID": "projection1", + "operatorProperties": { + "isDrop": false, + "attributes": [ + { + "originalAttribute": "column1", + "alias": "renamed_column1" + } + ] + }, + "customDisplayName": null + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + }, + { + "suggestion": "Add a Filter operator to filter data based on conditions.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "Filter", + "operatorID": "filter1", + "operatorProperties": { + "predicates": [ + { + "attribute": "your_attribute", + "condition": ">", + "value": "100" + } + ] + }, + "customDisplayName": null + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + }, + { + "suggestion": "Introduce an Aggregate operator to perform aggregation operations on data.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "Aggregate", + "operatorID": "aggregate1", + "operatorProperties": { + "aggregations": [ + { + "aggFunction": "sum", + "attribute": "your_numeric_attribute", + "result attribute": "sum_result" + } + ], + "groupByKeys": [ + "group_attribute" + ] + }, + "customDisplayName": null + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + } + ] +} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow2/by_path_interpretation.txt b/core/suggestion-service/test/results/workflow2/by_path_interpretation.txt new file mode 100644 index 00000000000..028fd72d9aa --- /dev/null +++ b/core/suggestion-service/test/results/workflow2/by_path_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260":{"operatorID":"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}},"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"text"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289":{"operatorID":"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289","operatorType":"PythonUDFV2","customDisplayName":"Python UDF","operatorProperties":{"code":"# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n","workers":1,"retainInputColumns":true,"outputColumns":[{"attributeName":"text_length","attributeType":"integer"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"text","attributeType":"string"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","portID":"output-0"},"target":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","portID":"output-0"},"target":{"operatorID":"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260","portID":"input-0"}},{"source":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","portID":"output-0"},"target":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","portID":"output-0"},"target":{"operatorID":"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289","portID":"input-0"}}]},"paths":[["CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260"],["CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289"]]} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow2/raw_interpretation.txt b/core/suggestion-service/test/results/workflow2/raw_interpretation.txt new file mode 100644 index 00000000000..166ee3ec825 --- /dev/null +++ b/core/suggestion-service/test/results/workflow2/raw_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260":{"operatorID":"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}},"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"text"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289":{"operatorID":"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289","operatorType":"PythonUDFV2","customDisplayName":"Python UDF","operatorProperties":{"code":"# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n","workers":1,"retainInputColumns":true,"outputColumns":[{"attributeName":"text_length","attributeType":"integer"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"text","attributeType":"string"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","portID":"output-0"},"target":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-b6799195-ecc4-426a-8e29-57e9248aa71f","portID":"output-0"},"target":{"operatorID":"Sort-operator-82fa7c7a-d1e1-45a6-8510-1f57c223e260","portID":"input-0"}},{"source":{"operatorID":"CSVFileScan-operator-aa00f100-e3fa-431c-9f2c-a03c845d01ef","portID":"output-0"},"target":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-5c723ccd-5f1a-4a60-8e0b-bc3cb7cbc8ac","portID":"output-0"},"target":{"operatorID":"PythonUDFV2-operator-29189f24-4d27-413f-9b67-1c8b37529289","portID":"input-0"}}]}} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow2/suggestions.json b/core/suggestion-service/test/results/workflow2/suggestions.json new file mode 100644 index 00000000000..76ba6f2443f --- /dev/null +++ b/core/suggestion-service/test/results/workflow2/suggestions.json @@ -0,0 +1,58 @@ +{ + "suggestions": [ + { + "suggestion": "Implement different recommendation models using machine learning techniques.", + "suggestionType": "improve", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "SklearnKNN", + "operatorID": "KNN-operator", + "operatorProperties": { + "target": "text_length", + "countVectorizer": true, + "text": "text", + "tfidfTransformer": true + }, + "customDisplayName": "KNN Model" + }, + { + "operatorType": "SklearnLogisticRegressionCV", + "operatorID": "LogisticRegressionCV-operator", + "operatorProperties": { + "target": "text_length", + "countVectorizer": true, + "text": "text", + "tfidfTransformer": true + }, + "customDisplayName": "Logistic Regression CV Model" + }, + { + "operatorType": "SklearnRandomForest", + "operatorID": "RandomForest-operator", + "operatorProperties": { + "target": "text_length", + "countVectorizer": true, + "text": "text", + "tfidfTransformer": true + }, + "customDisplayName": "Random Forest Model" + }, + { + "operatorType": "SklearnGradientBoosting", + "operatorID": "GradientBoosting-operator", + "operatorProperties": { + "target": "text_length", + "countVectorizer": true, + "text": "text", + "tfidfTransformer": true + }, + "customDisplayName": "Gradient Boosting Model" + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + } + ] +} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow3/by_path_interpretation.txt b/core/suggestion-service/test/results/workflow3/by_path_interpretation.txt new file mode 100644 index 00000000000..ccce3cf7315 --- /dev/null +++ b/core/suggestion-service/test/results/workflow3/by_path_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}},"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"text"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c":{"operatorID":"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c","operatorType":"PythonUDFV2","customDisplayName":"Python UDF","operatorProperties":{"code":"# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n","workers":1,"retainInputColumns":true,"outputColumns":[{"attributeName":"text_length","attributeType":"integer"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"text","attributeType":"string"}]}}}},"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865":{"operatorID":"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865","operatorType":"LineChart","customDisplayName":"Line Chart","operatorProperties":{"yLabel":"Y Axis","xLabel":"X Axis"},"error":{"type":"COMPILATION_ERROR","message":"java.lang.RuntimeException: Operator is not configured properly: null","details":"Stack trace for developers: \n\njava.lang.RuntimeException: Operator is not configured properly: null\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$9(WorkflowCompiler.scala:149)"},"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","portID":"output-0"},"target":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","portID":"output-0"},"target":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","portID":"input-0"}},{"source":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","portID":"output-0"},"target":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","portID":"output-0"},"target":{"operatorID":"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c","portID":"input-0"}},{"source":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","portID":"output-0"},"target":{"operatorID":"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865","portID":"input-0"}}]},"paths":[["CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865"],["CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c"]]} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow3/raw_interpretation.txt b/core/suggestion-service/test/results/workflow3/raw_interpretation.txt new file mode 100644 index 00000000000..1b7bcb0b641 --- /dev/null +++ b/core/suggestion-service/test/results/workflow3/raw_interpretation.txt @@ -0,0 +1 @@ +{"workflow":{"operators":{"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","operatorType":"CSVFileScan","customDisplayName":"CSV File Scan","operatorProperties":{"fileEncoding":"UTF_8","customDelimiter":",","hasHeader":true,"fileName":"/workflow_migrator@texera/ds4all-lecture-example/v1/clean_tweets.csv"},"error":null,"inputSchemas":{}},"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"tweet_id"},{"alias":"","originalAttribute":"create_at_month"},{"alias":"","originalAttribute":"favorite_count"},{"alias":"","originalAttribute":"retweet_count"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","operatorType":"Sort","customDisplayName":"Sort","operatorProperties":{"attributes":[{"attribute":"favorite_count","sortPreference":"DESC"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}},"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","operatorType":"Projection","customDisplayName":"Projection","operatorProperties":{"isDrop":false,"attributes":[{"alias":"","originalAttribute":"text"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"},{"attributeName":"user_id","attributeType":"integer"},{"attributeName":"text","attributeType":"string"},{"attributeName":"lang","attributeType":"string"},{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"is_retweet","attributeType":"boolean"}]}}}},"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c":{"operatorID":"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c","operatorType":"PythonUDFV2","customDisplayName":"Python UDF","operatorProperties":{"code":"# Choose from the following templates:\n# \nfrom pytexera import *\n\nclass ProcessTupleOperator(UDFOperatorV2):\n \n @overrides\n def process_tuple(self, tuple_: Tuple, port: int) -> Iterator[Optional[TupleLike]]:\n tuple_['text_length'] = len(tuple_['text'])\n yield tuple_\n\n# class ProcessBatchOperator(UDFBatchOperator):\n# BATCH_SIZE = 10 # must be a positive integer\n# \n# @overrides\n# def process_batch(self, batch: Batch, port: int) -> Iterator[Optional[BatchLike]]:\n# yield batch\n# \n# class ProcessTableOperator(UDFTableOperator):\n# \n# @overrides\n# def process_table(self, table: Table, port: int) -> Iterator[Optional[TableLike]]:\n# yield table\n","workers":1,"retainInputColumns":true,"outputColumns":[{"attributeName":"text_length","attributeType":"integer"}]},"error":null,"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"text","attributeType":"string"}]}}}},"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865":{"operatorID":"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865","operatorType":"LineChart","customDisplayName":"Line Chart","operatorProperties":{"yLabel":"Y Axis","xLabel":"X Axis"},"error":{"type":"COMPILATION_ERROR","message":"java.lang.RuntimeException: Operator is not configured properly: null","details":"Stack trace for developers: \n\njava.lang.RuntimeException: Operator is not configured properly: null\nedu.uci.ics.amber.compiler.WorkflowCompiler.$anonfun$expandLogicalPlan$9(WorkflowCompiler.scala:149)"},"inputSchemas":{"input-0":{"portID":"input-0","inputSchema":{"attributes":[{"attributeName":"create_at_month","attributeType":"timestamp"},{"attributeName":"favorite_count","attributeType":"integer"},{"attributeName":"tweet_id","attributeType":"integer"},{"attributeName":"retweet_count","attributeType":"integer"}]}}}}},"links":[{"source":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","portID":"output-0"},"target":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-fa6c76fa-3208-4f20-9b44-f920b91e898c","portID":"output-0"},"target":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","portID":"input-0"}},{"source":{"operatorID":"CSVFileScan-operator-7249c7e8-6d05-4572-931e-495d47f7ccf1","portID":"output-0"},"target":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","portID":"input-0"}},{"source":{"operatorID":"Projection-operator-9e443d52-a201-4f43-88fc-d8ad2cf4eb16","portID":"output-0"},"target":{"operatorID":"PythonUDFV2-operator-3e3c9f53-dae3-4dc4-b724-7ffdb8e7b80c","portID":"input-0"}},{"source":{"operatorID":"Sort-operator-3985eaf1-5af2-4f4a-bd0f-1c4b3f7e78c2","portID":"output-0"},"target":{"operatorID":"LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865","portID":"input-0"}}]}} \ No newline at end of file diff --git a/core/suggestion-service/test/results/workflow3/suggestions.json b/core/suggestion-service/test/results/workflow3/suggestions.json new file mode 100644 index 00000000000..6344ffd7cc7 --- /dev/null +++ b/core/suggestion-service/test/results/workflow3/suggestions.json @@ -0,0 +1,30 @@ +{ + "suggestions": [ + { + "suggestion": "Properly configure the LineChart operator with necessary parameters.", + "suggestionType": "fix", + "changes": { + "operatorsToAdd": [ + { + "operatorType": "LineChart", + "operatorID": "LineChart-operator-e3009841-32e4-4080-a6e4-f659762b3865", + "operatorProperties": { + "yLabel": "Y Axis", + "xLabel": "X Axis", + "lines": [ + { + "y": "favorite_count", + "x": "create_at_month", + "mode": "line with dots" + } + ] + }, + "customDisplayName": "Line Chart" + } + ], + "linksToAdd": [], + "operatorsToDelete": [] + } + } + ] +} \ No newline at end of file diff --git a/core/suggestion-service/workflow_interpretation/__init__.py b/core/suggestion-service/workflow_interpretation/__init__.py new file mode 100644 index 00000000000..2b4b2d5780b --- /dev/null +++ b/core/suggestion-service/workflow_interpretation/__init__.py @@ -0,0 +1,3 @@ +""" +Workflow Interpretation module for internal use in generating descriptions of workflows for suggestions. +""" diff --git a/core/suggestion-service/workflow_interpretation/interpreter.py b/core/suggestion-service/workflow_interpretation/interpreter.py new file mode 100644 index 00000000000..0d01e050fd3 --- /dev/null +++ b/core/suggestion-service/workflow_interpretation/interpreter.py @@ -0,0 +1,71 @@ +from model.llm.interpretation import ( + RawInterpretation, + PathInterpretation, + BaseInterpretation, + InterpretationMethod, +) +from model.texera.TexeraWorkflow import TexeraWorkflow +from typing import Dict, Any, Optional + +from model.web.input import CompilationStateInfo + + +class WorkflowInterpreter: + def __init__(self, method: InterpretationMethod): + self.method = method + pass + + def interpret_workflow( + self, + workflow: Dict[str, Any], + compilation_state_info: Optional[CompilationStateInfo] = None, + ) -> BaseInterpretation: + if compilation_state_info is None: + input_schema = {} + operator_errors = {} + else: + input_schema = { + op_id: [ + [attr.model_dump() for attr in schema] if schema else None + for schema in schemas + ] + for op_id, schemas in ( + compilation_state_info.operatorInputSchemaMap or {} + ).items() + } + operator_errors = compilation_state_info.operatorErrors or {} + if self.method == InterpretationMethod.RAW: + return self._interpret_raw(workflow, input_schema, operator_errors) + elif self.method == InterpretationMethod.BY_PATH: + return self._interpret_by_path(workflow, input_schema, operator_errors) + else: + raise ValueError(f"Unsupported interpretation method: {self.method}") + + def _interpret_raw( + self, + workflow: Dict[str, Any], + input_schema: Dict[str, Any], + operator_errors: Dict[str, Any], + ) -> RawInterpretation: + texera_workflow = TexeraWorkflow( + workflow_dict=workflow, + input_schema=input_schema, + operator_errors=operator_errors, + ) + return RawInterpretation( + workflow=texera_workflow.ToPydantic(), + ) + + def _interpret_by_path( + self, + workflow: Dict[str, Any], + input_schema: Dict[str, Any], + operator_errors: Dict[str, Any], + ) -> PathInterpretation: + texera_workflow = TexeraWorkflow( + workflow_dict=workflow, + input_schema=input_schema, + operator_errors=operator_errors, + ) + paths = texera_workflow.get_all_paths() + return PathInterpretation(workflow=texera_workflow.ToPydantic(), paths=paths) diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index 0876ce28513..3ed05a67719 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -181,5 +181,6 @@ libraryDependencies ++= Seq( "org.apache.commons" % "commons-vfs2" % "2.9.0", // for FileResolver throw VFS-related exceptions "io.lakefs" % "sdk" % "1.51.0", // for lakeFS api calls "com.typesafe" % "config" % "1.4.3", // config reader - "org.apache.commons" % "commons-jcs3-core" % "3.2" // Apache Commons JCS + "org.apache.commons" % "commons-jcs3-core" % "3.2", // Apache Commons JCS + "com.typesafe.play" %% "play-json" % "2.10.6", // json parsing library ) \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/AttributeTypeUtils.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/AttributeTypeUtils.scala index 2c660a14f50..0805d82bc9b 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/AttributeTypeUtils.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/AttributeTypeUtils.scala @@ -200,11 +200,21 @@ object AttributeTypeUtils extends Serializable { def parseTimestamp(fieldValue: Any): Timestamp = { val attempt: Try[Timestamp] = Try { fieldValue match { - case str: String => new Timestamp(DateParserUtils.parseDate(str.trim).getTime) - case long: java.lang.Long => new Timestamp(long) - case timestamp: Timestamp => timestamp - case date: java.util.Date => new Timestamp(date.getTime) - // Integer, Double, Boolean, Binary are considered to be illegal here. + case str: String => + new Timestamp(DateParserUtils.parseDate(str.trim).getTime) + + case ldt: java.time.LocalDateTime => + // Converts using the JVM default zone; change to a specific ZoneId if you need UTC + Timestamp.valueOf(ldt) + + case instant: java.time.Instant => + Timestamp.from(instant) + + case long: java.lang.Long => new Timestamp(long) + case ts: Timestamp => ts + case date: java.util.Date => new Timestamp(date.getTime) + + // Unsupported kinds fall through case _ => throw new AttributeTypeException( s"Unsupported type for parsing to Timestamp: ${fieldValue.getClass.getName}" @@ -219,7 +229,6 @@ object AttributeTypeUtils extends Serializable { e ) }.get - } @throws[AttributeTypeException] diff --git a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/LogicalOp.scala b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/LogicalOp.scala index b053ff929a9..11ede3a8e79 100644 --- a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/LogicalOp.scala +++ b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/LogicalOp.scala @@ -73,7 +73,7 @@ import edu.uci.ics.amber.operator.source.apis.twitter.v2.{ import edu.uci.ics.amber.operator.source.fetcher.URLFetcherOpDesc import edu.uci.ics.amber.operator.source.scan.FileScanSourceOpDesc import edu.uci.ics.amber.operator.source.scan.arrow.ArrowSourceOpDesc -import edu.uci.ics.amber.operator.source.scan.csv.CSVScanSourceOpDesc +import edu.uci.ics.amber.operator.source.scan.csv.{CSVScanSourceOpDesc, IntelligentCSVReaderOpDesc} import edu.uci.ics.amber.operator.source.scan.csvOld.CSVOldScanSourceOpDesc import edu.uci.ics.amber.operator.source.scan.json.JSONLScanSourceOpDesc import edu.uci.ics.amber.operator.source.scan.text.TextInputSourceOpDesc @@ -144,6 +144,7 @@ trait StateTransferFunc new Type(value = classOf[IfOpDesc], name = "If"), new Type(value = classOf[SankeyDiagramOpDesc], name = "SankeyDiagram"), new Type(value = classOf[IcicleChartOpDesc], name = "IcicleChart"), + new Type(value = classOf[IntelligentCSVReaderOpDesc], name = "IntelligentCSVReader"), new Type(value = classOf[CSVScanSourceOpDesc], name = "CSVFileScan"), // disabled the ParallelCSVScanSourceOpDesc so that it does not confuse user. it can be re-enabled when doing experiments. // new Type(value = classOf[ParallelCSVScanSourceOpDesc], name = "ParallelCSVFileScan"), diff --git a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/ScanSourceOpDesc.scala b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/ScanSourceOpDesc.scala index 42490188077..56606233ce9 100644 --- a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/ScanSourceOpDesc.scala +++ b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/ScanSourceOpDesc.scala @@ -39,6 +39,15 @@ abstract class ScanSourceOpDesc extends SourceOperatorDescriptor { @JsonIgnore var INFER_READ_LIMIT: Int = 100 + @JsonIgnore + var readAll: Boolean = true + + @JsonIgnore + var sampleEveryN: Int = 10 + + @JsonIgnore + var originalFileName: String = "" + @JsonProperty(required = true) @JsonSchemaTitle("File") @JsonDeserialize(contentAs = classOf[java.lang.String]) @@ -77,6 +86,7 @@ abstract class ScanSourceOpDesc extends SourceOperatorDescriptor { } def setResolvedFileName(uri: URI): Unit = { + originalFileName = fileName.get fileName = Some(uri.toASCIIString) } diff --git a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/csv/IntelligentCSVReaderOpDesc.scala b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/csv/IntelligentCSVReaderOpDesc.scala new file mode 100644 index 00000000000..de82f5f44e4 --- /dev/null +++ b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/source/scan/csv/IntelligentCSVReaderOpDesc.scala @@ -0,0 +1,196 @@ +/* + * 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. + */ + +package edu.uci.ics.amber.operator.source.scan.csv + +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty, JsonPropertyDescription} +import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle +import com.univocity.parsers.csv.{CsvFormat, CsvParser, CsvParserSettings} +import edu.uci.ics.amber.core.executor.OpExecWithCode +import edu.uci.ics.amber.core.storage.DocumentFactory +import edu.uci.ics.amber.core.tuple.AttributeTypeUtils.inferSchemaFromRows +import edu.uci.ics.amber.core.tuple.{AttributeType, Schema} +import edu.uci.ics.amber.core.virtualidentity.{ExecutionIdentity, WorkflowIdentity} +import edu.uci.ics.amber.core.workflow.{OutputPort, PhysicalOp, SchemaPropagationFunc} +import edu.uci.ics.amber.operator.metadata.{OperatorGroupConstants, OperatorInfo} +import edu.uci.ics.amber.operator.source.scan.ScanSourceOpDesc +import play.api.libs.json.{JsObject, JsString, Json} + +import java.io.InputStreamReader +import java.net.URI + +class IntelligentCSVReaderOpDesc extends ScanSourceOpDesc { + @JsonProperty(defaultValue = ",") + @JsonSchemaTitle("Delimiter") + @JsonPropertyDescription("delimiter to separate each line into fields") + @JsonInclude(JsonInclude.Include.NON_ABSENT) + var customDelimiter: Option[String] = None + + @JsonProperty(defaultValue = "true") + @JsonSchemaTitle("Header") + @JsonPropertyDescription("whether the CSV file contains a header line") + var hasHeader: Boolean = true + + @transient private var cachedSchema: Option[Schema] = None + + fileTypeName = Option("CSV") + + private def schemaToJson(s: Schema): String = { + val fields = s.getAttributeNames.zip(s.getAttributes).map { + case (name, attr) => name -> JsString(attr.getType.name()) + } + Json.stringify(JsObject(fields)) + } + + // ---------- main physical-plan construction ---------- + override def getPhysicalOp( + workflowId: WorkflowIdentity, + executionId: ExecutionIdentity + ): PhysicalOp = { + + // ▶︎ Generate the Python script on the fly + val code: String = generatePythonCode( + originalFileName, // full Texera dataset path + customDelimiter, // optional delimiter + hasHeader // header flag + ) + + val physicalOp = PhysicalOp + .sourcePhysicalOp(workflowId, executionId, operatorIdentifier, OpExecWithCode(code, "python")) + .withInputPorts(operatorInfo.inputPorts) + .withOutputPorts(operatorInfo.outputPorts) + .withIsOneToManyOp(true) + .withPropagateSchema( + SchemaPropagationFunc(_ => Map(operatorInfo.outputPorts.head.id -> sourceSchema())) + ) + .withLocationPreference(Option.empty) + .withParallelizable(false) // CSV reader is single-threaded for now + + physicalOp + } + + /** Creates a self-contained Python UDFSourceOperator script. */ + private def generatePythonCode( + fullPath: String, + delimiterOpt: Option[String], + header: Boolean + ): String = { + val schemaJson = schemaToJson(sourceSchema()) + val schemaJsonEsc = schemaJson.replace("\\", "\\\\").replace("\"", "\\\"") + + val sepKwarg = + delimiterOpt.filter(_ != ",").map(d => s"sep='$d'").getOrElse("") + val headerKwarg = + if (!header) "header=None" else "" + val kwargs = Seq(sepKwarg, headerKwarg).filter(_.nonEmpty).mkString(", ") + + val userJwtToken = + Option(System.getProperty("USER_JWT_TOKEN")) + .orElse(Option(System.getenv("USER_JWT_TOKEN"))) + .getOrElse("") + val tokenLine = + if (userJwtToken.nonEmpty) + s"""os.environ["USER_JWT_TOKEN"] = "${userJwtToken}"""" + else "" + + s""" + |from pytexera import * + |import os, json + | + |$tokenLine + | + |class GenerateOperator(UDFSourceOperator): + | + | def produce(self) -> Iterator[Union[TupleLike, TableLike, None]]: + | schema = json.loads("${schemaJsonEsc}") + | doc = DatasetFileDocument("$fullPath") + | table = doc.read_as_table(schema=schema, $kwargs) + | + | for _, row in table.iterrows(): + | yield row.to_dict() + |""".stripMargin + } + + // ------------- static operator metadata ------------------- + override def operatorInfo: OperatorInfo = { + OperatorInfo( + "Intelligent CSV Reader", + "User-defined (Python) CSV source that streams rows downstream", + OperatorGroupConstants.PYTHON_GROUP, + List.empty, // source → no inputs + List(OutputPort()), // single output + supportReconfiguration = true + ) + } + + override def sourceSchema(): Schema = { + cachedSchema.getOrElse { + val sch = computeSchema() // moved original body into helper + cachedSchema = Some(sch) + sch + } + } + + private def computeSchema(): Schema = { + if (customDelimiter.isEmpty || !fileResolved()) return null + + val stream = DocumentFactory.openReadonlyDocument(new URI(fileName.get)).asInputStream() + val inputReader = new InputStreamReader(stream, fileEncoding.getCharset) + + val csvFormat = new CsvFormat() + csvFormat.setDelimiter(customDelimiter.get.charAt(0)) + csvFormat.setLineSeparator("\n") + + val csvSetting = new CsvParserSettings() + csvSetting.setMaxCharsPerColumn(-1) + csvSetting.setFormat(csvFormat) + csvSetting.setHeaderExtractionEnabled(hasHeader) + csvSetting.setNullValue("") + + val parser = new CsvParser(csvSetting) + parser.beginParsing(inputReader) + + val dataBuf = scala.collection.mutable.ArrayBuffer.empty[Array[String]] + val maxRows = + if (readAll) Int.MaxValue else limit.getOrElse(INFER_READ_LIMIT).min(INFER_READ_LIMIT) + val step = math.max(sampleEveryN, 1) + + var row: Array[String] = null + var idx = 0 + while ({ row = parser.parseNext(); row != null && dataBuf.size < maxRows }) { + if (readAll || idx % step == 0) dataBuf += row + idx += 1 + } + parser.stopParsing() + inputReader.close() + + val attrTypes: Array[AttributeType] = + inferSchemaFromRows(dataBuf.iterator.asInstanceOf[Iterator[Array[Any]]]) + + val header: Array[String] = + if (hasHeader) + Option(parser.getContext.headers()) + .getOrElse((1 to attrTypes.length).map(i => s"column-$i").toArray) + else + (1 to attrTypes.length).map(i => s"column-$i").toArray + + header.indices.foldLeft(Schema()) { (schema, i) => + schema.add(header(i), attrTypes(i)) + } + } +} diff --git a/deployment/texera-web-application.dockerfile b/deployment/texera-web-application.dockerfile index f3464b6bba8..28722707c82 100644 --- a/deployment/texera-web-application.dockerfile +++ b/deployment/texera-web-application.dockerfile @@ -17,9 +17,6 @@ FROM node:18.17 AS build-gui -RUN apt-get update && apt-get install -y --no-install-recommends \ - python3 build-essential git ca-certificates - WORKDIR /gui COPY core/gui /gui RUN rm -f /gui/.yarnrc.yml