| <!doctype html> |
| <html lang="en" dir="ltr" class="docs-wrapper plugin-docs plugin-id-default docs-version-current docs-doc-page docs-doc-id-guide/examples" data-has-hydrated="false"> |
| <head> |
| <meta charset="UTF-8"> |
| <meta name="generator" content="Docusaurus v3.1.1"> |
| <title data-rh="true">Usage Examples | Apache Wayang (incubating)</title><meta data-rh="true" name="viewport" content="width=device-width,initial-scale=1"><meta data-rh="true" name="twitter:card" content="summary_large_image"><meta data-rh="true" property="og:url" content="https://wayang.apache.org/docs/guide/examples"><meta data-rh="true" property="og:locale" content="en"><meta data-rh="true" name="docusaurus_locale" content="en"><meta data-rh="true" name="docsearch:language" content="en"><meta data-rh="true" name="docusaurus_version" content="current"><meta data-rh="true" name="docusaurus_tag" content="docs-default-current"><meta data-rh="true" name="docsearch:version" content="current"><meta data-rh="true" name="docsearch:docusaurus_tag" content="docs-default-current"><meta data-rh="true" property="og:title" content="Usage Examples | Apache Wayang (incubating)"><meta data-rh="true" name="description" content="<!--"><meta data-rh="true" property="og:description" content="<!--"><link data-rh="true" rel="icon" href="/img/wayang-logo.jpg"><link data-rh="true" rel="canonical" href="https://wayang.apache.org/docs/guide/examples"><link data-rh="true" rel="alternate" href="https://wayang.apache.org/docs/guide/examples" hreflang="en"><link data-rh="true" rel="alternate" href="https://wayang.apache.org/docs/guide/examples" hreflang="x-default"><link rel="alternate" type="application/rss+xml" href="/blog/rss.xml" title="Apache Wayang (incubating) RSS Feed"> |
| <link rel="alternate" type="application/atom+xml" href="/blog/atom.xml" title="Apache Wayang (incubating) Atom Feed"><link rel="stylesheet" href="/assets/css/styles.af2ccd7c.css"> |
| <script src="/assets/js/runtime~main.535cf072.js" defer="defer"></script> |
| <script src="/assets/js/main.4ad3c7a5.js" defer="defer"></script> |
| </head> |
| <body class="navigation-with-keyboard"> |
| <script>!function(){function t(t){document.documentElement.setAttribute("data-theme",t)}var e=function(){try{return new URLSearchParams(window.location.search).get("docusaurus-theme")}catch(t){}}()||function(){try{return localStorage.getItem("theme")}catch(t){}}();t(null!==e?e:"light")}(),function(){try{const a=new URLSearchParams(window.location.search).entries();for(var[t,e]of a)if(t.startsWith("docusaurus-data-")){var n=t.replace("docusaurus-data-","data-");document.documentElement.setAttribute(n,e)}}catch(t){}}(),document.documentElement.setAttribute("data-announcement-bar-initially-dismissed",function(){try{return"true"===localStorage.getItem("docusaurus.announcement.dismiss")}catch(t){}return!1}())</script><div id="__docusaurus"><div role="region" aria-label="Skip to main content"><a class="skipToContent_fXgn" href="#__docusaurus_skipToContent_fallback">Skip to main content</a></div><div class="announcementBar_mb4j" style="background-color:#fafbfc;color:#091E42" role="banner"><div class="announcementBarPlaceholder_vyr4"></div><div class="content_knG7 announcementBarContent_xLdY">⭐️ If you like Apache Wayang, give it a star on <a target="_blank" href="https://github.com/apache/incubator-wayang">GitHub</a>! ⭐ </div><button type="button" aria-label="Close" class="clean-btn close closeButton_CVFx announcementBarClose_gvF7"><svg viewBox="0 0 15 15" width="14" height="14"><g stroke="currentColor" stroke-width="3.1"><path d="M.75.75l13.5 13.5M14.25.75L.75 14.25"></path></g></svg></button></div><nav aria-label="Main" class="navbar navbar--fixed-top"><div class="navbar__inner"><div class="navbar__items"><button aria-label="Toggle navigation bar" aria-expanded="false" class="navbar__toggle clean-btn" type="button"><svg width="30" height="30" viewBox="0 0 30 30" aria-hidden="true"><path stroke="currentColor" stroke-linecap="round" stroke-miterlimit="10" stroke-width="2" d="M4 7h22M4 15h22M4 23h22"></path></svg></button><a class="navbar__brand" href="/"><div class="navbar__logo"><img src="/img/wayang.png" alt="Wayang Logo" class="themedComponent_mlkZ themedComponent--light_NVdE"><img src="/img/wayang.png" alt="Wayang Logo" class="themedComponent_mlkZ themedComponent--dark_xIcU"></div><b class="navbar__title text--truncate"></b></a></div><div class="navbar__items navbar__items--right"><a class="navbar__item navbar__link" href="/docs/start/download">Download</a><a class="navbar__item navbar__link" href="/docs/introduction/about">About</a><a aria-current="page" class="navbar__item navbar__link navbar__link--active" href="/docs/guide/installation">Developers</a><div class="navbar__item dropdown dropdown--hoverable dropdown--right"><a href="#" aria-haspopup="true" aria-expanded="false" role="button" class="navbar__link">Community</a><ul class="dropdown__menu"><li><a class="dropdown__link" href="/blog/">Blog</a></li><li><a class="dropdown__link" href="/docs/community/mailinglist">Project</a></li></ul></div><div class="navbar__item dropdown dropdown--hoverable dropdown--right"><a href="#" aria-haspopup="true" aria-expanded="false" role="button" class="navbar__link">ASF</a><ul class="dropdown__menu"><li><a href="https://www.apache.org/" target="_blank" rel="noopener noreferrer" class="dropdown__link">Foundation</a></li><li><a href="https://www.apache.org/licenses/" target="_blank" rel="noopener noreferrer" class="dropdown__link">License</a></li><li><a href="https://www.apache.org/events/current-event.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Events</a></li><li><a href="https://privacy.apache.org/policies/privacy-policy-public.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Privacy</a></li><li><a href="https://www.apache.org/security/" target="_blank" rel="noopener noreferrer" class="dropdown__link">Security</a></li><li><a href="https://www.apache.org/foundation/sponsorship.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Sponsorship</a></li><li><a href="https://www.apache.org/foundation/thanks.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Thanks</a></li><li><a href="https://www.apache.org/foundation/policies/conduct.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Code of Conduct</a></li></ul></div><a href="https://github.com/apache/incubator-wayang" target="_blank" rel="noopener noreferrer" class="navbar__item navbar__link header-github-link" aria-label="GitHub repository"></a><div class="toggle_vylO colorModeToggle_DEke"><button class="clean-btn toggleButton_gllP toggleButtonDisabled_aARS" type="button" disabled="" title="Switch between dark and light mode (currently light mode)" aria-label="Switch between dark and light mode (currently light mode)" aria-live="polite"><svg viewBox="0 0 24 24" width="24" height="24" class="lightToggleIcon_pyhR"><path fill="currentColor" d="M12,9c1.65,0,3,1.35,3,3s-1.35,3-3,3s-3-1.35-3-3S10.35,9,12,9 M12,7c-2.76,0-5,2.24-5,5s2.24,5,5,5s5-2.24,5-5 S14.76,7,12,7L12,7z M2,13l2,0c0.55,0,1-0.45,1-1s-0.45-1-1-1l-2,0c-0.55,0-1,0.45-1,1S1.45,13,2,13z M20,13l2,0c0.55,0,1-0.45,1-1 s-0.45-1-1-1l-2,0c-0.55,0-1,0.45-1,1S19.45,13,20,13z M11,2v2c0,0.55,0.45,1,1,1s1-0.45,1-1V2c0-0.55-0.45-1-1-1S11,1.45,11,2z M11,20v2c0,0.55,0.45,1,1,1s1-0.45,1-1v-2c0-0.55-0.45-1-1-1C11.45,19,11,19.45,11,20z M5.99,4.58c-0.39-0.39-1.03-0.39-1.41,0 c-0.39,0.39-0.39,1.03,0,1.41l1.06,1.06c0.39,0.39,1.03,0.39,1.41,0s0.39-1.03,0-1.41L5.99,4.58z M18.36,16.95 c-0.39-0.39-1.03-0.39-1.41,0c-0.39,0.39-0.39,1.03,0,1.41l1.06,1.06c0.39,0.39,1.03,0.39,1.41,0c0.39-0.39,0.39-1.03,0-1.41 L18.36,16.95z M19.42,5.99c0.39-0.39,0.39-1.03,0-1.41c-0.39-0.39-1.03-0.39-1.41,0l-1.06,1.06c-0.39,0.39-0.39,1.03,0,1.41 s1.03,0.39,1.41,0L19.42,5.99z M7.05,18.36c0.39-0.39,0.39-1.03,0-1.41c-0.39-0.39-1.03-0.39-1.41,0l-1.06,1.06 c-0.39,0.39-0.39,1.03,0,1.41s1.03,0.39,1.41,0L7.05,18.36z"></path></svg><svg viewBox="0 0 24 24" width="24" height="24" class="darkToggleIcon_wfgR"><path fill="currentColor" d="M9.37,5.51C9.19,6.15,9.1,6.82,9.1,7.5c0,4.08,3.32,7.4,7.4,7.4c0.68,0,1.35-0.09,1.99-0.27C17.45,17.19,14.93,19,12,19 c-3.86,0-7-3.14-7-7C5,9.07,6.81,6.55,9.37,5.51z M12,3c-4.97,0-9,4.03-9,9s4.03,9,9,9s9-4.03,9-9c0-0.46-0.04-0.92-0.1-1.36 c-0.98,1.37-2.58,2.26-4.4,2.26c-2.98,0-5.4-2.42-5.4-5.4c0-1.81,0.89-3.42,2.26-4.4C12.92,3.04,12.46,3,12,3L12,3z"></path></svg></button></div><div class="navbarSearchContainer_Bca1"><div class="navbar__search"><span aria-label="expand searchbar" role="button" class="search-icon" tabindex="0"></span><input id="search_input_react" type="search" placeholder="Loading..." aria-label="Search" class="navbar__search-input search-bar" disabled=""></div></div></div></div><div role="presentation" class="navbar-sidebar__backdrop"></div></nav><div id="__docusaurus_skipToContent_fallback" class="main-wrapper mainWrapper_z2l0"><div class="docsWrapper_hBAB"><button aria-label="Scroll back to top" class="clean-btn theme-back-to-top-button backToTopButton_sjWU" type="button"></button><div class="docRoot_UBD9"><aside class="theme-doc-sidebar-container docSidebarContainer_YfHR"><div class="sidebarViewport_aRkj"><div class="sidebar_njMd"><nav aria-label="Docs sidebar" class="menu thin-scrollbar menu_SIkG menuWithAnnouncementBar_GW3s"><ul class="theme-doc-sidebar-menu menu__list"><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/installation">How to build Wayang</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/getting-started">Getting started</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/ml4all">Scalable Machine Learning</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/api-documentation">API documentation</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link menu__link--active" aria-current="page" href="/docs/guide/examples">Usage Examples</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/developing-in-wayang">Developing in Wayang</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/adding-operators">Adding an operator in Wayang</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/optimizer">Data processing optimizer</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/guide/wayang-docker">Using Wayang's docker image</a></li></ul></nav></div></div></aside><main class="docMainContainer_TBSr"><div class="container padding-top--md padding-bottom--lg"><div class="row"><div class="col docItemCol_VOVn"><div class="docItemContainer_Djhp"><article><nav class="theme-doc-breadcrumbs breadcrumbsContainer_Z_bl" aria-label="Breadcrumbs"><ul class="breadcrumbs" itemscope="" itemtype="https://schema.org/BreadcrumbList"><li class="breadcrumbs__item"><a aria-label="Home page" class="breadcrumbs__link" href="/"><svg viewBox="0 0 24 24" class="breadcrumbHomeIcon_YNFT"><path d="M10 19v-5h4v5c0 .55.45 1 1 1h3c.55 0 1-.45 1-1v-7h1.7c.46 0 .68-.57.33-.87L12.67 3.6c-.38-.34-.96-.34-1.34 0l-8.36 7.53c-.34.3-.13.87.33.87H5v7c0 .55.45 1 1 1h3c.55 0 1-.45 1-1z" fill="currentColor"></path></svg></a></li><li itemscope="" itemprop="itemListElement" itemtype="https://schema.org/ListItem" class="breadcrumbs__item breadcrumbs__item--active"><span class="breadcrumbs__link" itemprop="name">Usage Examples</span><meta itemprop="position" content="1"></li></ul></nav><div class="tocCollapsible_ETCw theme-doc-toc-mobile tocMobile_ITEo"><button type="button" class="clean-btn tocCollapsibleButton_TO0P">On this page</button></div><div class="theme-doc-markdown markdown"><h1>Guide to Development and Usage with Apache Wayang (incubating)</h1> |
| <p>This section provides a set of examples to illustrate how to use Apache Wayang for different tasks.</p> |
| <h2 class="anchor anchorWithStickyNavbar_LWe7" id="example-1-machine-learning-for-query-optimization-in-apache-wayang">Example 1: Machine Learning for query optimization in Apache Wayang<a href="#example-1-machine-learning-for-query-optimization-in-apache-wayang" class="hash-link" aria-label="Direct link to Example 1: Machine Learning for query optimization in Apache Wayang" title="Direct link to Example 1: Machine Learning for query optimization in Apache Wayang"></a></h2> |
| <p>Apache Wayang can be customized with concrete |
| implementations of the <code>EstimatableCost</code> interface in order to optimize |
| for a desired metric. The implementation can be enabled by providing it |
| to a <code>Configuration</code>.</p> |
| <div class="language-java codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#393A34;--prism-background-color:#f6f8fa"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-java codeBlock_bY9V thin-scrollbar" style="color:#393A34;background-color:#f6f8fa"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#393A34"><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">class</span><span class="token plain"> </span><span class="token class-name">CustomEstimatableCost</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">implements</span><span class="token plain"> </span><span class="token class-name">EstimatableCost</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/* Provide concrete implementations to match desired cost function(s)</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> * by implementing the interface in this class.</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> */</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"></span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"></span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">class</span><span class="token plain"> </span><span class="token class-name">WordCount</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">static</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">void</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">main</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">String</span><span class="token punctuation" style="color:#393A34">[</span><span class="token punctuation" style="color:#393A34">]</span><span class="token plain"> args</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/* Create a Wayang context and specify the platforms Wayang will consider */</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token plain"> config </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/* Provision of a EstimatableCost that implements the interface.*/</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> config</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">setCostModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">CustomEstimatableCost</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">WayangContext</span><span class="token plain"> wayangContext </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">WayangContext</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">config</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">withPlugin</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Java</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">basicPlugin</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">withPlugin</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Spark</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">basicPlugin</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/*... omitted */</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"></span><span class="token punctuation" style="color:#393A34">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div> |
| <p>In combination with an encoding scheme and a third party package to load |
| ML models, the following example shows how to predict runtimes of query |
| execution plans runtimes in Apache Wayang (incubating):</p> |
| <div class="language-java codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#393A34;--prism-background-color:#f6f8fa"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-java codeBlock_bY9V thin-scrollbar" style="color:#393A34;background-color:#f6f8fa"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#393A34"><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">class</span><span class="token plain"> </span><span class="token class-name">MLCost</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">implements</span><span class="token plain"> </span><span class="token class-name">EstimatableCost</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">EstimatableCostFactory</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getFactory</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">Factory</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">static</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">class</span><span class="token plain"> </span><span class="token class-name">Factory</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">implements</span><span class="token plain"> </span><span class="token class-name">EstimatableCostFactory</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">EstimatableCost</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">makeCost</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">MLCost</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">PlanImplementation</span><span class="token plain"> plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">boolean</span><span class="token plain"> isOverheadIncluded</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token plain"> config </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> plan</span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getOptimizationContext</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getConfiguration</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> model </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getInstance</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">config</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">ofExactly</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> model</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">runModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">OneHotEncoder</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">encode</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">catch</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Exception</span><span class="token plain"> e</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">zero</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getParallelEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">PlanImplementation</span><span class="token plain"> plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">boolean</span><span class="token plain"> isOverheadIncluded</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token plain"> config </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> plan</span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getOptimizationContext</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getConfiguration</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> model </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getInstance</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">config</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">ofExactly</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> model</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">runModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">OneHotEncoder</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">encode</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">catch</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Exception</span><span class="token plain"> e</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token class-name">ProbabilisticDoubleInterval</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">zero</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/** Returns a squashed cost estimate. */</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getSquashedEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">PlanImplementation</span><span class="token plain"> plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">boolean</span><span class="token plain"> isOverheadIncluded</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token plain"> config </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> plan</span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getOptimizationContext</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getConfiguration</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> model </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getInstance</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">config</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> model</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">runModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">OneHotEncoder</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">encode</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">catch</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Exception</span><span class="token plain"> e</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token number" style="color:#36acaa">0</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getSquashedParallelEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">PlanImplementation</span><span class="token plain"> plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">boolean</span><span class="token plain"> isOverheadIncluded</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Configuration</span><span class="token plain"> config </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> plan</span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getOptimizationContext</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getConfiguration</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> model </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getInstance</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">config</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> model</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">runModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">OneHotEncoder</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">encode</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">catch</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Exception</span><span class="token plain"> e</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token number" style="color:#36acaa">0</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token annotation punctuation" style="color:#393A34">@Override</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">Tuple</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">List</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">ProbabilisticDoubleInterval</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token generics punctuation" style="color:#393A34">,</span><span class="token generics"> </span><span class="token generics class-name">List</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Double</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getParallelOperatorJunctionAllCostEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">PlanImplementation</span><span class="token plain"> plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token class-name">Operator</span><span class="token plain"> operator</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">List</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">ProbabilisticDoubleInterval</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> intervalList </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">ArrayList</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">ProbabilisticDoubleInterval</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">List</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Double</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> doubleList </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">ArrayList</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Double</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> intervalList</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">add</span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token boolean" style="color:#36acaa">true</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> doubleList</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">add</span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getSquashedEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">plan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token boolean" style="color:#36acaa">true</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">Tuple</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">intervalList</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> doubleList</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token class-name">PlanImplementation</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">pickBestExecutionPlan</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Collection</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">PlanImplementation</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> executionPlans</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">ExecutionPlan</span><span class="token plain"> existingPlan</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Set</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Channel</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> openChannels</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">Set</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">ExecutionStage</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> executedStages</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">final</span><span class="token plain"> </span><span class="token class-name">PlanImplementation</span><span class="token plain"> bestPlanImplementation </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> executionPlans</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">stream</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">reduce</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">p1</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> p2</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token operator" style="color:#393A34">-></span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">final</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> t1 </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> p1</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getSquashedCostEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">final</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> t2 </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> p2</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getSquashedCostEstimate</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> t1 </span><span class="token operator" style="color:#393A34"><</span><span class="token plain"> t2 </span><span class="token operator" style="color:#393A34">?</span><span class="token plain"> p1 </span><span class="token operator" style="color:#393A34">:</span><span class="token plain"> p2</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">orElseThrow</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token operator" style="color:#393A34">-></span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">WayangException</span><span class="token punctuation" style="color:#393A34">(</span><span class="token string" style="color:#e3116c">"Could not find an execution plan."</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> bestPlanImplementation</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"></span><span class="token punctuation" style="color:#393A34">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div> |
| <p>Third-party packages such as <code>OnnxRuntime</code> can be used to load |
| pre-trained <code>.onnx</code> files that contain desired ML models.</p> |
| <div class="language-java codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#393A34;--prism-background-color:#f6f8fa"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-java codeBlock_bY9V thin-scrollbar" style="color:#393A34;background-color:#f6f8fa"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#393A34"><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">class</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">static</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> </span><span class="token constant" style="color:#36acaa">INSTANCE</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token class-name">OrtSession</span><span class="token plain"> session</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token class-name">OrtEnvironment</span><span class="token plain"> env</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">final</span><span class="token plain"> </span><span class="token class-name">Map</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">String</span><span class="token generics punctuation" style="color:#393A34">,</span><span class="token generics"> </span><span class="token generics class-name">OnnxTensor</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> inputMap </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">HashMap</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">final</span><span class="token plain"> </span><span class="token class-name">Set</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">String</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> requestedOutputs </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">HashSet</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">static</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">getInstance</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Configuration</span><span class="token plain"> configuration</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">throws</span><span class="token plain"> </span><span class="token class-name">OrtException</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">if</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token constant" style="color:#36acaa">INSTANCE</span><span class="token plain"> </span><span class="token operator" style="color:#393A34">==</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">null</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token constant" style="color:#36acaa">INSTANCE</span><span class="token plain"> </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">configuration</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token constant" style="color:#36acaa">INSTANCE</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">private</span><span class="token plain"> </span><span class="token class-name">OrtMLModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Configuration</span><span class="token plain"> configuration</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">throws</span><span class="token plain"> </span><span class="token class-name">OrtException</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">loadModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">configuration</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getStringProperty</span><span class="token punctuation" style="color:#393A34">(</span><span class="token string" style="color:#e3116c">"wayang.ml.model.file"</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">void</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">loadModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">String</span><span class="token plain"> filePath</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">throws</span><span class="token plain"> </span><span class="token class-name">OrtException</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">if</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">env </span><span class="token operator" style="color:#393A34">==</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">null</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">env </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OrtEnvironment</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getEnvironment</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">if</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">session </span><span class="token operator" style="color:#393A34">==</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">null</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">session </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> env</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">createSession</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">filePath</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">new</span><span class="token plain"> </span><span class="token class-name">OrtSession</span><span class="token class-name punctuation" style="color:#393A34">.</span><span class="token class-name">SessionOptions</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">void</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">closeSession</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">throws</span><span class="token plain"> </span><span class="token class-name">OrtException</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">session</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">close</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">env</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">close</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token comment" style="color:#999988;font-style:italic">/**</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> * @param encodedVector</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> * @return NaN on error, and a predicted cost on any other value.</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> * @throws OrtException</span><br></span><span class="token-line" style="color:#393A34"><span class="token comment" style="color:#999988;font-style:italic"> */</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">public</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> </span><span class="token function" style="color:#d73a49">runModel</span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Vector</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Long</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> encodedVector</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">throws</span><span class="token plain"> </span><span class="token class-name">OrtException</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">double</span><span class="token plain"> costPrediction</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">OnnxTensor</span><span class="token plain"> tensor </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token class-name">OnnxTensor</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">createTensor</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">env</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> encodedVector</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">inputMap</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">put</span><span class="token punctuation" style="color:#393A34">(</span><span class="token string" style="color:#e3116c">"input"</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> tensor</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">this</span><span class="token punctuation" style="color:#393A34">.</span><span class="token plain">requestedOutputs</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">add</span><span class="token punctuation" style="color:#393A34">(</span><span class="token string" style="color:#e3116c">"output"</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token class-name">BiFunction</span><span class="token generics punctuation" style="color:#393A34"><</span><span class="token generics class-name">Result</span><span class="token generics punctuation" style="color:#393A34">,</span><span class="token generics"> </span><span class="token generics class-name">String</span><span class="token generics punctuation" style="color:#393A34">,</span><span class="token generics"> </span><span class="token generics class-name">Double</span><span class="token generics punctuation" style="color:#393A34">></span><span class="token plain"> unwrapFunc </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">r</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> s</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token operator" style="color:#393A34">-></span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">(</span><span class="token keyword" style="color:#00009f">double</span><span class="token punctuation" style="color:#393A34">[</span><span class="token punctuation" style="color:#393A34">]</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> r</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">get</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">s</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">get</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">getValue</span><span class="token punctuation" style="color:#393A34">(</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">[</span><span class="token number" style="color:#36acaa">0</span><span class="token punctuation" style="color:#393A34">]</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"> </span><span class="token keyword" style="color:#00009f">catch</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">OrtException</span><span class="token plain"> e</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> </span><span class="token class-name">Double</span><span class="token class-name punctuation" style="color:#393A34">.</span><span class="token class-name">NaN</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">try</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">(</span><span class="token class-name">Result</span><span class="token plain"> r </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> session</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">run</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">inputMap</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> requestedOutputs</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">)</span><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> costPrediction </span><span class="token operator" style="color:#393A34">=</span><span class="token plain"> unwrapFunc</span><span class="token punctuation" style="color:#393A34">.</span><span class="token function" style="color:#d73a49">apply</span><span class="token punctuation" style="color:#393A34">(</span><span class="token plain">r</span><span class="token punctuation" style="color:#393A34">,</span><span class="token plain"> </span><span class="token string" style="color:#e3116c">"output"</span><span class="token punctuation" style="color:#393A34">)</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain" style="display:inline-block"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token keyword" style="color:#00009f">return</span><span class="token plain"> costPrediction</span><span class="token punctuation" style="color:#393A34">;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"> </span><span class="token punctuation" style="color:#393A34">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#393A34"><span class="token plain"></span><span class="token punctuation" style="color:#393A34">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div></div></article><nav class="pagination-nav docusaurus-mt-lg" aria-label="Docs pages"><a class="pagination-nav__link pagination-nav__link--prev" href="/docs/guide/api-documentation"><div class="pagination-nav__sublabel">Previous</div><div class="pagination-nav__label">API documentation</div></a><a class="pagination-nav__link pagination-nav__link--next" href="/docs/guide/developing-in-wayang"><div class="pagination-nav__sublabel">Next</div><div class="pagination-nav__label">Developing in Wayang</div></a></nav></div></div><div class="col col--3"><div class="tableOfContents_bqdL thin-scrollbar theme-doc-toc-desktop"><ul class="table-of-contents table-of-contents__left-border"><li><a href="#example-1-machine-learning-for-query-optimization-in-apache-wayang" class="table-of-contents__link toc-highlight">Example 1: Machine Learning for query optimization in Apache Wayang</a></li></ul></div></div></div></div></main></div></div></div><footer class="footer footer--dark"><div class="container container-fluid"><div class="row footer__links"><div class="col footer__col"><div class="footer__title">Community</div><ul class="footer__items clean-list"><li class="footer__item"><a href="https://lists.apache.org/list.html?dev@wayang.apache.org" target="_blank" rel="noopener noreferrer" class="footer__link-item">Mailing list<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li class="footer__item"><a href="https://www.youtube.com/@apachewayang" target="_blank" rel="noopener noreferrer" class="footer__link-item">YouTube<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li class="footer__item"><a href="https://www.linkedin.com/company/apachewayang" target="_blank" rel="noopener noreferrer" class="footer__link-item">LinkedIn<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li class="footer__item"><a href="https://www.reddit.com/r/ApacheWayang" target="_blank" rel="noopener noreferrer" class="footer__link-item">Reddit<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li class="footer__item"><a href="https://twitter.com/apachewayang" target="_blank" rel="noopener noreferrer" class="footer__link-item">Twitter<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li></ul></div><div class="col footer__col"><div class="footer__title">Documentation</div><ul class="footer__items clean-list"><li class="footer__item"><a class="footer__link-item" href="/docs/start/download">Install</a></li><li class="footer__item"><a class="footer__link-item" href="/docs/introduction/features">Features</a></li><li class="footer__item"><a class="footer__link-item" href="/docs/introduction/benchmark">Benchmark</a></li><li class="footer__item"><a class="footer__link-item" href="/docs/community/security">Security</a></li></ul></div><div class="col footer__col"><div class="footer__title">Repositories</div><ul class="footer__items clean-list"><li class="footer__item"><a href="https://github.com/apache/incubator-wayang" target="_blank" rel="noopener noreferrer" class="footer__link-item">Wayang<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li class="footer__item"><a href="https://github.com/apache/incubator-wayang-website" target="_blank" rel="noopener noreferrer" class="footer__link-item">Website<svg width="13.5" height="13.5" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li></ul></div></div><div class="footer__bottom text--center"><div class="margin-bottom--sm"><a href="https://incubator.apache.org/" rel="noopener noreferrer" class="footerLogoLink_BH7S"><img src="/img/apache-incubator.svg" alt="Apache Incubator logo" class="footer__logo themedComponent_mlkZ themedComponent--light_NVdE" width="200"><img src="/img/apache-incubator.svg" alt="Apache Incubator logo" class="footer__logo themedComponent_mlkZ themedComponent--dark_xIcU" width="200"></a></div><div class="footer__copyright"><div> |
| <p> Apache Wayang is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF. </p> |
| <p> |
| Copyright © 2025 The Apache Software Foundation, Licensed under the Apache License, Version 2.0. <br> |
| Apache Wayang, Wayang, Apache, the Apache feather logo, and the Apache Wayang project logo are either registered trademarks or trademarks of The Apache Software Foundation in the United States and other countries. All other marks mentioned may be trademarks or registered trademarks of their respective owners. |
| </p> |
| </div></div></div></div></footer></div> |
| </body> |
| </html> |