diff --git a/plugins/ui/DESIGN.md b/plugins/ui/DESIGN.md
index fbbfd49c3..ee91b496e 100644
--- a/plugins/ui/DESIGN.md
+++ b/plugins/ui/DESIGN.md
@@ -1964,31 +1964,35 @@ The above examples are all in Python, and particularly take some advantage of la
##### Rendering
-When you call a function decorated by `@ui.component`, it will return a `UiNode` object that has a reference to the function it is decorated; that is to say, the function does _not_ get run immediately. The function is only run when the `UiNode` is rendered by the client, and the result is sent back to the client. This allows the `@ui.component` decorator to execute the function with the appropriate rendering context, and also allows for memoization of the function (e.g. if the function is called multiple times with the same arguments, it will only be executed once - akin to a [memoized component](https://react.dev/reference/react/memo) or PureComponent in React).
+When you call a function decorated by `@ui.component`, it will return an `Element` object that has a reference to the function it is decorated; that is to say, the function does _not_ get run immediately. The function is only run when the `Element` is rendered by the client, and the result is sent back to the client. This allows the `@ui.component` decorator to execute the function with the appropriate rendering context. The client must also set the initial state before rendering, allowing the client to persist the state and re-render in the future.
Let's say we execute the following, where a table is filtered based on the value of a text input:
```python
+from deephaven import ui
+
+
@ui.component
def text_filter_table(source, column, initial_value=""):
- value, set_value = use_state(initial_value)
- ti = ui.text_field(value, on_change=set_value)
+ value, set_value = ui.use_state(initial_value)
+ ti = ui.text_field(value=value, on_change=set_value)
tt = source.where(f"{column}=`{value}`")
return [ti, tt]
+# This will render two panels, one filtering the table by Sym, and the other by Exchange
@ui.component
-def sym_exchange(source):
- tft1 = text_filter_table(source, "Sym")
- tft2 = text_filter_table(source, "Exchange")
- return ui.flex(tft1, tft2, direction="row")
+def double_text_filter_table(source):
+ tft1 = text_filter_table(source, "sym")
+ tft2 = text_filter_table(source, "exchange")
+ return ui.panel(tft1, title="Sym"), ui.panel(tft2, title="Exchange")
import deephaven.plot.express as dx
-t = dx.data.stocks()
+_stocks = dx.data.stocks()
-tft = text_filter_table(t, "sym")
+tft = double_text_filter_table(_stocks)
```
Which should result in a UI like this:
@@ -2013,21 +2017,21 @@ sequenceDiagram
W->>UIP: Open tft
UIP->>C: Export tft
- C-->>UIP: tft (UiNode)
+ C-->>UIP: tft (Element)
- Note over UIP: UI knows about object tft
sym_exchange not executed yet
+ Note over UIP: UI knows about object tft
double_text_filter_table not executed yet
- UIP->>SP: Render tft
- SP->>SP: Run sym_exchange
- Note over SP: sym_exchange executes, running text_filter_table twice
- SP-->>UIP: Result (document=flex([tft1, tft2]), exported_objects=[tft1, tft2])
+ UIP->>SP: Render tft (initialState)
+ SP->>SP: Run double_text_filter_table
+ Note over SP: double_text_filter_table executes, running text_filter_table twice
+ SP-->>UIP: Result (document=[panel(tft1), pane(tft2)], exported_objects=[tft1, tft2])
UIP-->>W: Display Result
U->>UIP: Change text input 1
UIP->>SP: Change state
- SP->>SP: Run sym_exchange
- Note over SP: sym_exchange executes, text_filter_table only
runs once for the one changed input
only exports the new table, as client already has previous tables
- SP-->>UIP: Result (document=flex([tft1', tft2], exported_objects=[tft1']))
+ SP->>SP: Run double_text_filter_table
+ Note over SP: double_text_filter_table executes, text_filter_table only
runs once for the one changed input
only exports the new table, as client already has previous tables
+ SP-->>UIP: Result (document=[panel(tft1'), panel(tft2)], state={}, exported_objects=[tft1'])
UIP-->>W: Display Result
```
@@ -2040,14 +2044,15 @@ sequenceDiagram
participant UIP as UI Plugin
participant SP as Server Plugin
- UIP->>SP: obj.getDataAsString()
- Note over UIP, SP: Uses json-rpc
- SP-->>UIP: documentUpdated(Document)
+ Note over UIP, SP: Uses JSON-RPC
+ UIP->>SP: setState(initialState)
+ SP-->>UIP: documentUpdated(Document, State)
loop Callback
UIP->>SP: foo(params)
SP-->>UIP: foo result
- SP->>UIP: documentUpdated(Document)
+ SP->>UIP: documentUpdated(Document, State)
+ Note over UIP: Client can store State to restore the same state later
end
```
diff --git a/plugins/ui/src/deephaven/ui/_internal/RenderContext.py b/plugins/ui/src/deephaven/ui/_internal/RenderContext.py
index ea4a9a4d2..28e89c298 100644
--- a/plugins/ui/src/deephaven/ui/_internal/RenderContext.py
+++ b/plugins/ui/src/deephaven/ui/_internal/RenderContext.py
@@ -5,13 +5,13 @@
from typing import (
Any,
Callable,
+ Dict,
Optional,
TypeVar,
Union,
Generator,
Generic,
cast,
- Set,
)
from functools import partial
from deephaven import DHError
@@ -48,11 +48,16 @@
A function that takes the old value and returns the new value for a state.
"""
-ContextKey = Union[str, int]
+ContextKey = str
"""
The key for a child context.
"""
+ChildrenContextDict = Dict[ContextKey, "RenderContext"]
+"""
+The child contexts for a RenderContext.
+"""
+
@dataclass
class ValueWithLiveness(Generic[T]):
@@ -62,6 +67,17 @@ class ValueWithLiveness(Generic[T]):
liveness_scope: Union[LivenessScope, None]
+ContextState = Dict[StateKey, ValueWithLiveness[Any]]
+"""
+The state for a context.
+"""
+
+ExportedRenderState = Dict[str, Any]
+"""
+The serializable state of a RenderContext. Used to serialize the state for the client.
+"""
+
+
def _value_or_call(
value: T | None | Callable[[], T | None]
) -> ValueWithLiveness[T | None]:
@@ -83,6 +99,19 @@ def _value_or_call(
return ValueWithLiveness(value=value, liveness_scope=None)
+def _should_retain_value(value: ValueWithLiveness[T | None]) -> bool:
+ """
+ Determine if the given value should be retained by the current context.
+
+ Args:
+ value: The value to check.
+
+ Returns:
+ True if the value should be retained, False otherwise.
+ """
+ return value.liveness_scope is None and isinstance(value.value, (str, int, float))
+
+
_local_data = threading.local()
@@ -133,12 +162,12 @@ class RenderContext:
Count of hooks used in the render. Should only be set after initial render.
"""
- _state: dict[StateKey, ValueWithLiveness[Any]]
+ _state: ContextState
"""
The state for this context.
"""
- _children_context: dict[ContextKey, "RenderContext"]
+ _children_context: ChildrenContextDict
"""
The child contexts for this context.
"""
@@ -354,3 +383,52 @@ def manage(self, liveness_scope: LivenessScope) -> None:
"""
assert self is get_context()
self._collected_scopes.add(cast(LivenessScope, liveness_scope.j_scope))
+
+ def export_state(self) -> ExportedRenderState:
+ """
+ Export the state of this context. This is used to serialize the state for the client.
+
+ Returns:
+ The exported serializable state of this context.
+ """
+ exported_state: ExportedRenderState = {}
+
+ # We need to iterate through all of our state and export anything that doesn't have a LivenessScope right now (anything serializable)
+ def retained_values(state: ContextState):
+ for key, value in state.items():
+ if _should_retain_value(value):
+ yield key, value.value
+
+ if len(state := dict(retained_values(self._state))) > 0:
+ exported_state["state"] = state
+
+ # Now iterate through all the children contexts, and only include them in the export if they're not empty
+ def retained_children(children: ChildrenContextDict):
+ for key, child in children.items():
+ if len(child_state := child.export_state()) > 0:
+ yield key, child_state
+
+ if len(children_state := dict(retained_children(self._children_context))) > 0:
+ exported_state["children"] = children_state
+
+ return exported_state
+
+ def import_state(self, state: dict[str, Any]) -> None:
+ """
+ Import the state of this context. This is used to deserialize the state from the client.
+
+ Args:
+ state: The state to import.
+ """
+ self._state.clear()
+ self._children_context.clear()
+ if "state" in state:
+ for key, value in state["state"].items():
+ # When python dict is converted to JSON, all keys are converted to strings. We convert them back to int here.
+ self._state[int(key)] = ValueWithLiveness(
+ value=value, liveness_scope=None
+ )
+ if "children" in state:
+ for key, child_state in state["children"].items():
+ self.get_child_context(key).import_state(child_state)
+ logger.debug("New state is %s", self._state)
diff --git a/plugins/ui/src/deephaven/ui/_internal/__init__.py b/plugins/ui/src/deephaven/ui/_internal/__init__.py
index b618b0466..88a725643 100644
--- a/plugins/ui/src/deephaven/ui/_internal/__init__.py
+++ b/plugins/ui/src/deephaven/ui/_internal/__init__.py
@@ -8,6 +8,7 @@
get_context,
NoContextException,
ValueWithLiveness,
+ ExportedRenderState,
)
from .utils import (
get_component_name,
diff --git a/plugins/ui/src/deephaven/ui/object_types/ElementMessageStream.py b/plugins/ui/src/deephaven/ui/object_types/ElementMessageStream.py
index 3efd1ca93..5d4d1c593 100644
--- a/plugins/ui/src/deephaven/ui/object_types/ElementMessageStream.py
+++ b/plugins/ui/src/deephaven/ui/object_types/ElementMessageStream.py
@@ -16,7 +16,7 @@
from .._internal import wrap_callable
from ..elements import Element
from ..renderer import NodeEncoder, Renderer, RenderedNode
-from .._internal import RenderContext, StateUpdateCallable
+from .._internal import RenderContext, StateUpdateCallable, ExportedRenderState
logger = logging.getLogger(__name__)
@@ -132,7 +132,7 @@ def __init__(self, element: Element, connection: MessageStream):
self._connection = connection
self._message_id = 0
self._manager = JSONRPCResponseManager()
- self._dispatcher = Dispatcher()
+ self._dispatcher = self._make_dispatcher()
self._encoder = NodeEncoder(separators=(",", ":"))
self._context = RenderContext(self._queue_state_update, self._queue_callable)
self._renderer = Renderer(self._context)
@@ -155,40 +155,43 @@ def _render(self) -> None:
try:
node = self._renderer.render(self._element)
+ state = self._context.export_state()
+ self._send_document_update(node, state)
except Exception as e:
logger.exception("Error rendering %s", self._element.name)
raise e
- self._send_document_update(node)
-
def _process_callable_queue(self) -> None:
"""
Process any queued callables, then re-renders the element if it is dirty.
"""
- with self._exec_context:
- with self._render_lock:
- self._render_thread = threading.current_thread()
- self._render_state = _RenderState.RENDERING
-
- while not self._callable_queue.empty():
- item = self._callable_queue.get()
- with liveness_scope():
- try:
- item()
- except Exception as e:
- logger.exception(e)
-
- if self._is_dirty:
- self._render()
-
- with self._render_lock:
- self._render_thread = None
- if not self._callable_queue.empty() or self._is_dirty:
- # There are still callables to process, so queue up another render
- self._render_state = _RenderState.QUEUED
- submit_task("concurrent", self._process_callable_queue)
- else:
- self._render_state = _RenderState.IDLE
+ try:
+ with self._exec_context:
+ with self._render_lock:
+ self._render_thread = threading.current_thread()
+ self._render_state = _RenderState.RENDERING
+
+ while not self._callable_queue.empty():
+ item = self._callable_queue.get()
+ with liveness_scope():
+ try:
+ item()
+ except Exception as e:
+ logger.exception(e)
+
+ if self._is_dirty:
+ self._render()
+
+ with self._render_lock:
+ self._render_thread = None
+ if not self._callable_queue.empty() or self._is_dirty:
+ # There are still callables to process, so queue up another render
+ self._render_state = _RenderState.QUEUED
+ submit_task("concurrent", self._process_callable_queue)
+ else:
+ self._render_state = _RenderState.IDLE
+ except Exception as e:
+ logger.exception(e)
def _mark_dirty(self) -> None:
"""
@@ -232,9 +235,9 @@ def _queue_callable(self, callable: Callable[[], None]) -> None:
def start(self) -> None:
"""
- Start the message stream. This will start the render loop and queue up the initial render.
+ Start the message stream. All we do is send a blank message to start. Client will respond with the initial state.
"""
- self._mark_dirty()
+ self._connection.on_data(b"", [])
def on_close(self) -> None:
pass
@@ -302,12 +305,31 @@ def _make_request(self, method: str, *params: Any) -> dict[str, Any]:
"id": self._get_next_message_id(),
}
- def _send_document_update(self, root: RenderedNode) -> None:
+ def _make_dispatcher(self) -> Dispatcher:
+ dispatcher = Dispatcher()
+ dispatcher["setState"] = self._set_state
+ return dispatcher
+
+ def _set_state(self, state: ExportedRenderState) -> None:
+ """
+ Set the state of the element. This is called by the client on initial load.
+
+ Args:
+ state: The state to set
+ """
+ logger.debug("Setting state: %s", state)
+ self._context.import_state(state)
+ self._mark_dirty()
+
+ def _send_document_update(
+ self, root: RenderedNode, state: ExportedRenderState
+ ) -> None:
"""
Send a document update to the client. Currently just sends the entire document for each update.
Args:
root: The root node of the document to send
+ state: The state of the node to preserve
"""
# TODO(#67): Send a diff of the document instead of the entire document.
@@ -316,11 +338,16 @@ def _send_document_update(self, root: RenderedNode) -> None:
new_objects = encoder_result["new_objects"]
callable_id_dict = encoder_result["callable_id_dict"]
- request = self._make_notification("documentUpdated", encoded_document)
+ logger.debug("Exported state: %s", state)
+ encoded_state = json.dumps(state)
+
+ request = self._make_notification(
+ "documentUpdated", encoded_document, encoded_state
+ )
payload = json.dumps(request)
logger.debug(f"Sending payload: {payload}")
- dispatcher = Dispatcher()
+ dispatcher = self._make_dispatcher()
for callable, callable_id in callable_id_dict.items():
logger.debug("Registering callable %s", callable_id)
dispatcher[callable_id] = wrap_callable(callable)
diff --git a/plugins/ui/src/deephaven/ui/object_types/ElementType.py b/plugins/ui/src/deephaven/ui/object_types/ElementType.py
index b677839ac..02d07ad8e 100644
--- a/plugins/ui/src/deephaven/ui/object_types/ElementType.py
+++ b/plugins/ui/src/deephaven/ui/object_types/ElementType.py
@@ -17,7 +17,11 @@ def name(self) -> str:
def is_type(self, obj: Any) -> bool:
return isinstance(obj, Element)
- def create_client_connection(self, obj: Element, connection: MessageStream):
+ def create_client_connection(
+ self, obj: object, connection: MessageStream
+ ) -> MessageStream:
+ if not isinstance(obj, Element):
+ raise TypeError(f"Expected Element, got {type(obj)}")
client_connection = ElementMessageStream(obj, connection)
client_connection.start()
return client_connection
diff --git a/plugins/ui/src/deephaven/ui/renderer/Renderer.py b/plugins/ui/src/deephaven/ui/renderer/Renderer.py
index 218b53f0f..4052b9b28 100644
--- a/plugins/ui/src/deephaven/ui/renderer/Renderer.py
+++ b/plugins/ui/src/deephaven/ui/renderer/Renderer.py
@@ -47,7 +47,7 @@ def _render_list(
"""
logger.debug("_render_list %s", item)
return [
- _render_item(value, context.get_child_context(key))
+ _render_item(value, context.get_child_context(str(key)))
for key, value in enumerate(item)
]
diff --git a/plugins/ui/src/js/src/DashboardPlugin.tsx b/plugins/ui/src/js/src/DashboardPlugin.tsx
index 9c8373fb6..33cad703e 100644
--- a/plugins/ui/src/js/src/DashboardPlugin.tsx
+++ b/plugins/ui/src/js/src/DashboardPlugin.tsx
@@ -24,6 +24,7 @@ import { useDebouncedCallback } from '@deephaven/react-hooks';
import styles from './styles.scss?inline';
import {
ReadonlyWidgetData,
+ WidgetDataUpdate,
WidgetFetch,
WidgetId,
} from './widget/WidgetTypes';
@@ -194,12 +195,14 @@ export function DashboardPlugin(
});
// We may need to clean up some panels for this widget if it hasn't actually loaded yet
// We should be able to always be able to do this even if it does load, so just remove any panels from the initial load
- const { openWidgets } = initialPluginData;
- const openWidget = openWidgets?.[panelId];
- if (openWidget?.data?.panelIds != null) {
- const { panelIds } = openWidget.data;
- for (let i = 0; i < panelIds.length; i += 1) {
- LayoutUtils.closeComponent(layout.root, { id: panelIds[i] });
+ if (initialPluginData != null) {
+ const { openWidgets } = initialPluginData;
+ const openWidget = openWidgets?.[panelId];
+ if (openWidget?.data?.panelIds != null) {
+ const { panelIds } = openWidget.data;
+ for (let i = 0; i < panelIds.length; i += 1) {
+ LayoutUtils.closeComponent(layout.root, { id: panelIds[i] });
+ }
}
}
},
@@ -258,7 +261,7 @@ export function DashboardPlugin(
);
const handleWidgetDataChange = useCallback(
- (widgetId: string, data: ReadonlyWidgetData) => {
+ (widgetId: string, data: WidgetDataUpdate) => {
log.debug('handleWidgetDataChange', widgetId, data);
setWidgetMap(prevWidgetMap => {
const newWidgetMap = new Map(prevWidgetMap);
@@ -268,7 +271,10 @@ export function DashboardPlugin(
}
newWidgetMap.set(widgetId, {
...oldWidget,
- data,
+ data: {
+ ...oldWidget.data,
+ ...data,
+ },
});
return newWidgetMap;
});
diff --git a/plugins/ui/src/js/src/widget/DashboardWidgetHandler.tsx b/plugins/ui/src/js/src/widget/DashboardWidgetHandler.tsx
index 26660e569..5ad3ad41b 100644
--- a/plugins/ui/src/js/src/widget/DashboardWidgetHandler.tsx
+++ b/plugins/ui/src/js/src/widget/DashboardWidgetHandler.tsx
@@ -5,7 +5,7 @@ import React, { useCallback } from 'react';
import { WidgetDescriptor } from '@deephaven/dashboard';
import { Widget } from '@deephaven/jsapi-types';
import Log from '@deephaven/log';
-import { ReadonlyWidgetData, WidgetId } from './WidgetTypes';
+import { ReadonlyWidgetData, WidgetDataUpdate, WidgetId } from './WidgetTypes';
import WidgetHandler from './WidgetHandler';
const log = Log.module('@deephaven/js-plugin-ui/DashboardWidgetHandler');
@@ -27,7 +27,7 @@ export interface DashboardWidgetHandlerProps {
onClose?: (widgetId: WidgetId) => void;
/** Triggered when the data in the widget changes */
- onDataChange?: (widgetId: WidgetId, data: ReadonlyWidgetData) => void;
+ onDataChange?: (widgetId: WidgetId, data: WidgetDataUpdate) => void;
}
function DashboardWidgetHandler({
@@ -42,7 +42,7 @@ function DashboardWidgetHandler({
}, [onClose, id]);
const handleDataChange = useCallback(
- (data: ReadonlyWidgetData) => {
+ (data: WidgetDataUpdate) => {
log.debug('handleDataChange', id, data);
onDataChange?.(id, data);
},
diff --git a/plugins/ui/src/js/src/widget/DocumentHandler.tsx b/plugins/ui/src/js/src/widget/DocumentHandler.tsx
index 4cd2485a6..fddf8d7a2 100644
--- a/plugins/ui/src/js/src/widget/DocumentHandler.tsx
+++ b/plugins/ui/src/js/src/widget/DocumentHandler.tsx
@@ -5,7 +5,11 @@ import Log from '@deephaven/log';
import { EMPTY_FUNCTION } from '@deephaven/utils';
import { ReactPanelManagerContext } from '../layout/ReactPanelManager';
import { getRootChildren } from './DocumentUtils';
-import { ReadonlyWidgetData, WidgetData } from './WidgetTypes';
+import {
+ ReadonlyWidgetData,
+ WidgetData,
+ WidgetDataUpdate,
+} from './WidgetTypes';
const log = Log.module('@deephaven/js-plugin-ui/DocumentHandler');
@@ -22,7 +26,7 @@ export type DocumentHandlerProps = React.PropsWithChildren<{
initialData?: ReadonlyWidgetData;
/** Triggered when the data in the document changes */
- onDataChange?: (data: ReadonlyWidgetData) => void;
+ onDataChange?: (data: WidgetDataUpdate) => void;
/** Triggered when all panels opened from this document have closed */
onClose?: () => void;
diff --git a/plugins/ui/src/js/src/widget/WidgetHandler.tsx b/plugins/ui/src/js/src/widget/WidgetHandler.tsx
index 0e2e1052e..455498a48 100644
--- a/plugins/ui/src/js/src/widget/WidgetHandler.tsx
+++ b/plugins/ui/src/js/src/widget/WidgetHandler.tsx
@@ -25,7 +25,11 @@ import {
isElementNode,
isObjectNode,
} from '../elements/ElementUtils';
-import { ReadonlyWidgetData, WidgetMessageEvent } from './WidgetTypes';
+import {
+ ReadonlyWidgetData,
+ WidgetDataUpdate,
+ WidgetMessageEvent,
+} from './WidgetTypes';
import DocumentHandler from './DocumentHandler';
import { getComponentForElement } from './WidgetUtils';
@@ -44,8 +48,8 @@ export interface WidgetHandlerProps {
/** Triggered when all panels opened from this widget have closed */
onClose?: () => void;
- /** Triggered when the data in the widget changes */
- onDataChange?: (data: ReadonlyWidgetData) => void;
+ /** Triggered when the data in the widget changes. Only the changed data is provided. */
+ onDataChange?: (data: WidgetDataUpdate) => void;
}
function WidgetHandler({
@@ -53,10 +57,11 @@ function WidgetHandler({
onDataChange = EMPTY_FUNCTION,
fetch,
widget: descriptor,
- initialData,
+ initialData: initialDataProp,
}: WidgetHandlerProps) {
const [widget, setWidget] = useState();
const [document, setDocument] = useState();
+ const [initialData] = useState(initialDataProp);
// When we fetch a widget, the client is then responsible for the exported objects.
// These objects could stay alive even after the widget is closed if we wanted to,
@@ -169,64 +174,100 @@ function WidgetHandler({
}
log.debug('Adding methods to jsonClient');
- jsonClient.addMethod('documentUpdated', async (params: [string]) => {
- log.debug2('documentUpdated', params[0]);
- const newDocument = parseDocument(params[0]);
- setDocument(newDocument);
- });
+ jsonClient.addMethod(
+ 'documentUpdated',
+ async (params: [string, string]) => {
+ log.debug2('documentUpdated', params);
+ const [documentParam, stateParam] = params;
+ const newDocument = parseDocument(documentParam);
+ setDocument(newDocument);
+ if (stateParam != null) {
+ try {
+ const newState = JSON.parse(stateParam);
+ onDataChange({ state: newState });
+ } catch (e) {
+ log.warn(
+ 'Error parsing state, widget state may not be persisted.',
+ e
+ );
+ }
+ }
+ }
+ );
return () => {
jsonClient.rejectAllPendingRequests('Widget was changed');
};
},
- [jsonClient, parseDocument]
+ [jsonClient, onDataChange, parseDocument]
);
- useEffect(() => {
- if (widget == null) {
- return;
- }
- // Need to reset the exported object map and count
- const widgetExportedObjectMap = new Map();
- exportedObjectMap.current = widgetExportedObjectMap;
- exportedObjectCount.current = 0;
- function receiveData(
- data: string,
- newExportedObjects: WidgetExportedObject[]
- ) {
- log.debug2('Data received', data, newExportedObjects);
- updateExportedObjects(newExportedObjects);
- jsonClient?.receiveAndSend(JSON.parse(data));
- }
+ /**
+ * Triggered when the widget object is loaded. Initializes the state of the widget and/or receives initial data.
+ */
+ useEffect(
+ function initializeWidget() {
+ if (widget == null || jsonClient == null) {
+ return;
+ }
+ // Need to reset the exported object map and count
+ const widgetExportedObjectMap = new Map();
+ exportedObjectMap.current = widgetExportedObjectMap;
+ exportedObjectCount.current = 0;
- const cleanup = widget.addEventListener(
- // This is defined as dh.Widget.EVENT_MESSAGE in Core, but that constant doesn't exist on the Enterprise API
- // Dashboard plugins in Enterprise are loaded with the Enterprise API in the context of the dashboard, so trying to fetch the constant fails
- // Just use the constant value here instead. Another option would be to add the Widget constants to Enterprise, but we don't want to port over all that functionality.
- 'message',
- (event: WidgetMessageEvent) => {
- receiveData(
- event.detail.getDataAsString(),
- event.detail.exportedObjects
- );
+ // Set a var to the client that we know will not be null in the closure below
+ const activeClient = jsonClient;
+ function receiveData(
+ data: string,
+ newExportedObjects: WidgetExportedObject[]
+ ) {
+ log.debug2('Data received', data, newExportedObjects);
+ updateExportedObjects(newExportedObjects);
+ if (data.length > 0) {
+ activeClient.receiveAndSend(JSON.parse(data));
+ }
}
- );
- log.debug('Receiving initial data');
- // We need to get the initial data and process it. It should be a documentUpdated command.
- receiveData(widget.getDataAsString(), widget.exportedObjects);
+ const cleanup = widget.addEventListener(
+ // This is defined as dh.Widget.EVENT_MESSAGE in Core, but that constant doesn't exist on the Enterprise API
+ // Dashboard plugins in Enterprise are loaded with the Enterprise API in the context of the dashboard, so trying to fetch the constant fails
+ // Just use the constant value here instead. Another option would be to add the Widget constants to Enterprise, but we don't want to port over all that functionality.
+ 'message',
+ (event: WidgetMessageEvent) => {
+ receiveData(
+ event.detail.getDataAsString(),
+ event.detail.exportedObjects
+ );
+ }
+ );
+
+ log.debug('Receiving initial data');
+ // We need to get the initial data and process it. If it's an old version of the plugin, it could be a documentUpdated command.
+ receiveData(widget.getDataAsString(), widget.exportedObjects);
- return () => {
- log.debug('Cleaning up widget', widget);
- cleanup();
- widget.close();
+ // We set the initial state of the widget. We'll then get a documentUpdated as a response.
+ activeClient.request('setState', [initialData?.state ?? {}]).then(
+ result => {
+ log.debug('Set state result', result);
+ },
+ e => {
+ log.error('Error setting initial state: ', e);
+ }
+ );
- // Clean up any exported objects that haven't been closed yet
- Array.from(widgetExportedObjectMap.values()).forEach(exportedObject => {
- exportedObject.close();
- });
- };
- }, [jsonClient, parseDocument, updateExportedObjects, widget]);
+ return () => {
+ log.debug('Cleaning up widget', widget);
+ cleanup();
+ widget.close();
+
+ // Clean up any exported objects that haven't been closed yet
+ Array.from(widgetExportedObjectMap.values()).forEach(exportedObject => {
+ exportedObject.close();
+ });
+ };
+ },
+ [jsonClient, initialData, parseDocument, updateExportedObjects, widget]
+ );
useEffect(
function loadWidget() {
diff --git a/plugins/ui/src/js/src/widget/WidgetTypes.ts b/plugins/ui/src/js/src/widget/WidgetTypes.ts
index f56548dce..4ab78403c 100644
--- a/plugins/ui/src/js/src/widget/WidgetTypes.ts
+++ b/plugins/ui/src/js/src/widget/WidgetTypes.ts
@@ -15,6 +15,12 @@ export type WidgetFetch = (takeOwnership?: boolean) => Promise;
export type WidgetData = {
/** Panel IDs that are opened by this widget */
panelIds?: string[];
+
+ /** State of the widget on the Python side */
+ state?: Record;
};
export type ReadonlyWidgetData = Readonly;
+
+/** Contains an update for widget data. Only the keys that are updated are passed. */
+export type WidgetDataUpdate = Partial;
diff --git a/plugins/ui/test/deephaven/ui/test_render.py b/plugins/ui/test/deephaven/ui/test_render.py
index ef544a23b..1992b4e82 100644
--- a/plugins/ui/test/deephaven/ui/test_render.py
+++ b/plugins/ui/test/deephaven/ui/test_render.py
@@ -119,3 +119,134 @@ def test_context(self):
self.assertEqual(child_context1.get_state(0), 3)
# Shouldn't have triggered a change
self.assertEqual(on_change.call_count, 2)
+
+
+class RenderExportTestCase(BaseTestCase):
+ def test_export_empty_context(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+
+ with rc.open():
+ pass
+
+ state = rc.export_state()
+ self.assertEqual(state, {})
+
+ def test_export_basic_state(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+
+ with rc.open():
+ rc.init_state(0, 1)
+ rc.init_state(1, 2)
+ rc.init_state(2, 3)
+
+ state = rc.export_state()
+ self.assertEqual(state, {"state": {0: 1, 1: 2, 2: 3}})
+
+ def test_export_nested_state(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+
+ with rc.open():
+ rc.init_state(0, 1)
+ child_context0 = rc.get_child_context(0)
+ with child_context0.open():
+ child_context0.init_state(0, 2)
+ child_context0.init_state(1, 3)
+ child_context1 = child_context0.get_child_context(0)
+ with child_context1.open():
+ child_context1.init_state(0, 4)
+ child_context1.init_state(1, 5)
+
+ state = rc.export_state()
+ self.assertEqual(
+ state,
+ {
+ "state": {0: 1},
+ "children": {
+ 0: {"state": {0: 2, 1: 3}, "children": {0: {"state": {0: 4, 1: 5}}}}
+ },
+ },
+ )
+
+ def test_ignore_empty_state(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+
+ with rc.open():
+ rc.init_state(0, 1)
+ rc.init_state(1, 2)
+ rc.init_state(2, 3)
+ rc.set_state(0, None)
+ rc.set_state(1, None)
+ rc.set_state(2, None)
+
+ child_context0 = rc.get_child_context(0)
+ with child_context0.open():
+ child_context1 = child_context0.get_child_context(0)
+ with child_context1.open():
+ child_context1.init_state(0, None)
+
+ state = rc.export_state()
+ self.assertEqual(state, {})
+
+
+class RenderImportTestCase(BaseTestCase):
+ def test_import_empty_context(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ on_change = Mock(side_effect=lambda x: x())
+ rc = make_render_context(on_change)
+
+ # Empty context should reset the state if there was one
+ with rc.open():
+ rc.init_state(0, 2)
+ self.assertEqual(rc.has_state(0), True)
+ self.assertEqual(rc.get_state(0), 2)
+
+ state = {}
+ rc.import_state(state)
+ with rc.open():
+ self.assertEqual(rc.has_state(0), False)
+
+ def test_import_basic_state(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+ state = {"state": {0: 3}}
+ rc.import_state(state)
+ with rc.open():
+ self.assertEqual(rc.has_state(0), True)
+ self.assertEqual(rc.get_state(0), 3)
+
+ def test_import_nested_state(self):
+ from deephaven.ui._internal.RenderContext import RenderContext
+
+ rc = make_render_context()
+ state = {
+ "state": {0: 1},
+ "children": {
+ 0: {"state": {0: 2, 1: 3}, "children": {0: {"state": {0: 4, 1: 5}}}}
+ },
+ }
+ rc.import_state(state)
+ with rc.open():
+ self.assertEqual(rc.has_state(0), True)
+ self.assertEqual(rc.get_state(0), 1)
+ child_context0 = rc.get_child_context(0)
+ with child_context0.open():
+ self.assertEqual(child_context0.has_state(0), True)
+ self.assertEqual(child_context0.get_state(0), 2)
+ self.assertEqual(child_context0.has_state(1), True)
+ self.assertEqual(child_context0.get_state(1), 3)
+ child_context1 = child_context0.get_child_context(0)
+ with child_context1.open():
+ self.assertEqual(child_context1.has_state(0), True)
+ self.assertEqual(child_context1.get_state(0), 4)
+ self.assertEqual(child_context1.has_state(1), True)
+ self.assertEqual(child_context1.get_state(1), 5)